diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index a182cd7..a5bfe41 100644
--- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -533,6 +533,7 @@
HIVE_ORC_INCLUDE_FILE_FOOTER_IN_SPLITS("hive.orc.splits.include.file.footer", false),
HIVE_ORC_CACHE_STRIPE_DETAILS_SIZE("hive.orc.cache.stripe.details.size", 10000),
HIVE_ORC_COMPUTE_SPLITS_NUM_THREADS("hive.orc.compute.splits.num.threads", 10),
+ HIVE_ORC_SKIP_CORRUPT_DATA("hive.exec.orc.skip.corrupt.data", false),
HIVESKEWJOIN("hive.optimize.skewjoin", false),
HIVECONVERTJOIN("hive.auto.convert.join", true),
@@ -594,6 +595,10 @@
HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false), // try to use sorted merge bucket map join
HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true),
HIVEOPTREDUCEDEDUPLICATIONMINREDUCER("hive.optimize.reducededuplication.min.reducer", 4),
+ // when enabled dynamic partitioning column will be globally sorted.
+ // this way we can keep only one record writer open for each partition value
+ // in the reducer thereby reducing the memory pressure on reducer
+ HIVEOPTSORTDYNAMICPARTITION("hive.optimize.sort.dynamic.partition", true),
HIVESAMPLINGFORORDERBY("hive.optimize.sampling.orderby", false),
HIVESAMPLINGNUMBERFORORDERBY("hive.optimize.sampling.orderby.number", 1000),
diff --git conf/hive-default.xml.template conf/hive-default.xml.template
index 0d08aa2..fc85a1c 100644
--- conf/hive-default.xml.template
+++ conf/hive-default.xml.template
@@ -1032,6 +1032,17 @@
+ hive.optimize.sort.dynamic.partition
+ true
+
+ Enabling this flag will sort dynamic partitioning columns (and bucket
+ column if present). This enables reducers to keep only one record writer open at any
+ time thereby reducing memory pressure on the reducers. Enabling this flag will enforce
+ sorting and bucketing (for bucketed tables).
+
+
+
+
hive.enforce.sortmergebucketmapjoin
false
If the user asked for sort-merge bucketed map-side join, and it cannot be performed,
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index fb0b772..630015b 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -47,6 +47,7 @@
import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc.DPSortState;
import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
import org.apache.hadoop.hive.ql.plan.PlanUtils;
import org.apache.hadoop.hive.ql.plan.SkewedColumnPositionPair;
@@ -92,6 +93,9 @@
protected transient boolean statsCollectRawDataSize;
private transient boolean[] statsFromRecordWriter;
private transient boolean isCollectRWStats;
+ private transient int numColsToRead;
+ private transient FSPaths prevFsp;
+ private transient String prevPath;
/**
* RecordWriter.
@@ -317,6 +321,16 @@ protected void initializeOp(Configuration hconf) throws HiveException {
lbSetup();
}
+ numColsToRead = numDynParts;
+
+ // bucket columns on FileSinkDesc is set by hive.optimize.sort.dynamic.partition
+ // optimization. It adds bucket number as last column of value field.
+ if (conf.getBucketCols() != null && !conf.getBucketCols().isEmpty()) {
+ numColsToRead += 1;
+ dpVals = new ArrayList(numColsToRead);
+ dpWritables = new ArrayList(numColsToRead);
+ }
+
if (!bDynParts) {
fsp = new FSPaths(specPath);
@@ -367,7 +381,7 @@ private void dpSetup() {
this.dpStartCol = 0;
for (StructField sf : fieldOI) {
String fn = sf.getFieldName();
- if (!dpCtx.getInputToDPCols().containsKey(fn)) {
+ if (!dpCtx.getInputToDPCols().containsKey(fn) && !fn.equalsIgnoreCase("_bcol0")) {
newFieldsOI.add(sf.getFieldObjectInspector());
newFieldsName.add(sf.getFieldName());
this.dpStartCol++;
@@ -422,56 +436,7 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException {
bucketMap.put(bucketNum, filesIdx);
taskId = Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), bucketNum);
}
- if (isNativeTable) {
- fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, null);
- LOG.info("Final Path: FS " + fsp.finalPaths[filesIdx]);
- fsp.outPaths[filesIdx] = fsp.getTaskOutPath(taskId);
- LOG.info("Writing to temp file: FS " + fsp.outPaths[filesIdx]);
- } else {
- fsp.finalPaths[filesIdx] = fsp.outPaths[filesIdx] = specPath;
- }
- try {
- // The reason to keep these instead of using
- // OutputFormat.getRecordWriter() is that
- // getRecordWriter does not give us enough control over the file name that
- // we create.
- String extension = Utilities.getFileExtension(jc, isCompressed,
- hiveOutputFormat);
- if (!bDynParts && !this.isSkewedStoredAsSubDirectories) {
- fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, parent, extension);
- } else {
- fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, extension);
- }
-
- } catch (Exception e) {
- e.printStackTrace();
- throw new HiveException(e);
- }
- LOG.info("New Final Path: FS " + fsp.finalPaths[filesIdx]);
-
- if (isNativeTable) {
- try {
- // in recent hadoop versions, use deleteOnExit to clean tmp files.
- autoDelete = fs.deleteOnExit(fsp.outPaths[filesIdx]);
- } catch (IOException e) {
- throw new HiveException(e);
- }
- }
-
- Utilities.copyTableJobPropertiesToConf(conf.getTableInfo(), jc);
- // only create bucket files only if no dynamic partitions,
- // buckets of dynamic partitions will be created for each newly created partition
- fsp.outWriters[filesIdx] = HiveFileFormatUtils.getHiveRecordWriter(
- jc, conf.getTableInfo(), outputClass, conf, fsp.outPaths[filesIdx],
- reporter);
- // If the record writer provides stats, get it from there instead of the serde
- statsFromRecordWriter[filesIdx] = fsp.outWriters[filesIdx] instanceof StatsProvidingRecordWriter;
- // increment the CREATED_FILES counter
- if (reporter != null) {
- reporter.incrCounter(HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP),
- Operator.HIVECOUNTERCREATEDFILES,
- 1);
- }
+ createBucketForFileIdx(fsp, filesIdx);
filesIdx++;
}
assert filesIdx == numFiles;
@@ -480,8 +445,6 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException {
if (isNativeTable) {
autoDelete = fs.deleteOnExit(fsp.outPaths[0]);
}
- } catch (HiveException e) {
- throw e;
} catch (Exception e) {
e.printStackTrace();
throw new HiveException(e);
@@ -490,6 +453,52 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException {
filesCreated = true;
}
+ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) throws HiveException {
+ try {
+ if (isNativeTable) {
+ fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, null);
+ LOG.info("Final Path: FS " + fsp.finalPaths[filesIdx]);
+ fsp.outPaths[filesIdx] = fsp.getTaskOutPath(taskId);
+ LOG.info("Writing to temp file: FS " + fsp.outPaths[filesIdx]);
+ } else {
+ fsp.finalPaths[filesIdx] = fsp.outPaths[filesIdx] = specPath;
+ }
+ // The reason to keep these instead of using
+ // OutputFormat.getRecordWriter() is that
+ // getRecordWriter does not give us enough control over the file name that
+ // we create.
+ String extension = Utilities.getFileExtension(jc, isCompressed, hiveOutputFormat);
+ if (!bDynParts && !this.isSkewedStoredAsSubDirectories) {
+ fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, parent, extension);
+ } else {
+ fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, extension);
+ }
+
+ LOG.info("New Final Path: FS " + fsp.finalPaths[filesIdx]);
+
+ if (isNativeTable) {
+ // in recent hadoop versions, use deleteOnExit to clean tmp files.
+ autoDelete = fs.deleteOnExit(fsp.outPaths[filesIdx]);
+ }
+
+ Utilities.copyTableJobPropertiesToConf(conf.getTableInfo(), jc);
+ // only create bucket files only if no dynamic partitions,
+ // buckets of dynamic partitions will be created for each newly created partition
+ fsp.outWriters[filesIdx] = HiveFileFormatUtils.getHiveRecordWriter(jc, conf.getTableInfo(),
+ outputClass, conf, fsp.outPaths[filesIdx], reporter);
+ // If the record writer provides stats, get it from there instead of the serde
+ statsFromRecordWriter[filesIdx] = fsp.outWriters[filesIdx] instanceof StatsProvidingRecordWriter;
+ // increment the CREATED_FILES counter
+ if (reporter != null) {
+ reporter.incrCounter(HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP),
+ Operator.HIVECOUNTERCREATEDFILES, 1);
+ }
+
+ } catch (IOException e) {
+ throw new HiveException(e);
+ }
+ }
+
/**
* Report status to JT so that JT won't kill this task if closing takes too long
* due to too many files to close and the NN is overloaded.
@@ -535,7 +544,7 @@ public void processOp(Object row, int tag) throws HiveException {
// copy the DP column values from the input row to dpVals
dpVals.clear();
dpWritables.clear();
- ObjectInspectorUtils.partialCopyToStandardObject(dpWritables, row, dpStartCol, numDynParts,
+ ObjectInspectorUtils.partialCopyToStandardObject(dpWritables, row, dpStartCol, numColsToRead,
(StructObjectInspector) inputObjInspectors[0], ObjectInspectorCopyOption.WRITABLE);
// get a set of RecordWriter based on the DP column values
// pass the null value along to the escaping process to determine what the dir should be
@@ -647,8 +656,10 @@ private FSPaths createNewPaths(String dirName) throws HiveException {
fsp2.taskOutputTempPath =
new Path(fsp2.taskOutputTempPath, dirName);
}
- createBucketFiles(fsp2);
- valToPaths.put(dirName, fsp2);
+ if(!conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED)) {
+ createBucketFiles(fsp2);
+ valToPaths.put(dirName, fsp2);
+ }
return fsp2;
}
@@ -705,9 +716,16 @@ protected FSPaths getDynOutPaths(List row, String lbDirName) throws Hive
// get the path corresponding to the dynamic partition columns,
String dpDir = getDynPartDirectory(row, dpColNames, numDynParts);
+ String pathKey = null;
if (dpDir != null) {
- dpDir = appendListBucketingDirName(lbDirName, dpDir);
- FSPaths fsp2 = valToPaths.get(dpDir);
+ dpDir = appendDirName(lbDirName, dpDir);
+ pathKey = dpDir;
+ if(conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED)) {
+ String buckNum = dpVals.get(dpVals.size() - 1);
+ taskId = Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), buckNum);
+ pathKey = appendDirName(taskId, dpDir);
+ }
+ FSPaths fsp2 = valToPaths.get(pathKey);
if (fsp2 == null) {
// check # of dp
@@ -717,7 +735,45 @@ protected FSPaths getDynOutPaths(List row, String lbDirName) throws Hive
ErrorMsg.DYNAMIC_PARTITIONS_TOO_MANY_PER_NODE_ERROR.getErrorCodedMsg()
+ "Maximum was set to: " + maxPartitions);
}
+
+ if (!conf.getDpSortState().equals(DPSortState.NONE) && prevFsp != null) {
+ // close the previous fsp as it is no longer needed
+ prevFsp.closeWriters(false);
+
+ // since we are closing the previous fsp's record writers, we need to see if we can get
+ // stats from the record writer and store in the previous fsp that is cached
+ if (conf.isGatherStats() && isCollectRWStats) {
+ FSPaths cachedFsp = valToPaths.get(prevPath);
+ for (int idx = 0; idx < cachedFsp.outWriters.length; idx++) {
+ FSRecordWriter outWriter = cachedFsp.outWriters[idx];
+ if (outWriter != null) {
+ SerDeStats stats = ((StatsProvidingRecordWriter) outWriter).getStats();
+ if (stats != null) {
+ cachedFsp.stat.addToStat(StatsSetupConst.RAW_DATA_SIZE, stats.getRawDataSize());
+ cachedFsp.stat.addToStat(StatsSetupConst.ROW_COUNT, stats.getRowCount());
+ }
+ }
+ }
+ }
+
+ // let writers release the memory for garbage collection
+ for (int i = 0; i < prevFsp.outWriters.length; i++) {
+ prevFsp.outWriters[i] = null;
+ }
+
+ prevFsp = null;
+ }
+
fsp2 = createNewPaths(dpDir);
+ if (prevFsp == null) {
+ prevFsp = fsp2;
+ prevPath = pathKey;
+ }
+
+ if(conf.getDpSortState().equals(DPSortState.PARTITION_BUCKET_SORTED)) {
+ createBucketForFileIdx(fsp2, 0);
+ valToPaths.put(pathKey, fsp2);
+ }
}
fp = fsp2;
} else {
@@ -727,17 +783,16 @@ protected FSPaths getDynOutPaths(List row, String lbDirName) throws Hive
}
/**
- * Append list bucketing dir name to original dir name.
- * Skewed columns cannot be partitioned columns.
- * @param lbDirName
- * @param dpDir
+ * Append dir to source dir
+ * @param appendDir
+ * @param srcDir
* @return
*/
- private String appendListBucketingDirName(String lbDirName, String dpDir) {
- StringBuilder builder = new StringBuilder(dpDir);
- dpDir = (lbDirName == null) ? dpDir : builder.append(Path.SEPARATOR).append(lbDirName)
+ private String appendDirName(String appendDir, String srcDir) {
+ StringBuilder builder = new StringBuilder(srcDir);
+ srcDir = (appendDir == null) ? srcDir : builder.append(Path.SEPARATOR).append(appendDir)
.toString();
- return dpDir;
+ return srcDir;
}
// given the current input row, the mapping for input col info to dp columns, and # of dp cols,
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
index c378dc7..45c4724 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
@@ -36,17 +36,25 @@
import org.apache.hadoop.hive.ql.plan.api.OperatorType;
import org.apache.hadoop.hive.serde2.SerDeException;
import org.apache.hadoop.hive.serde2.Serializer;
+import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
+import org.apache.hadoop.hive.serde2.lazy.LazyFactory;
+import org.apache.hadoop.hive.serde2.lazy.LazyObject;
+import org.apache.hadoop.hive.serde2.lazy.LazyPrimitive;
+import org.apache.hadoop.hive.serde2.lazy.LazyString;
import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector.StandardUnion;
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
import org.apache.hadoop.io.BinaryComparable;
import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.OutputCollector;
/**
@@ -67,13 +75,28 @@
* The evaluators for the value columns. Value columns are passed to reducer
* in the "value".
*/
- protected transient ExprNodeEvaluator[] valueEval;
+ protected transient ExprNodeEvaluator[] outputValueEval;
+ /**
+ * This is used by hive.optimize.sort.dynamic.partition case where bucket
+ * number is appended to value. outputValueEval will have one column more than
+ * inputValueEval.
+ */
+ protected transient ExprNodeEvaluator[] inputValueEval;
/**
* The evaluators for the partition columns (CLUSTER BY or DISTRIBUTE BY in
* Hive language). Partition columns decide the reducer that the current row
* goes to. Partition columns are not passed to reducer.
*/
protected transient ExprNodeEvaluator[] partitionEval;
+ /**
+ * Evaluators for bucketing columns. This is used to compute bucket number.
+ */
+ protected transient ExprNodeEvaluator[] bucketEval = null;
+ /**
+ * OI for outputValueEval. Since new column is added to value new OI is
+ * required.
+ */
+ protected transient ArrayList objectInspectors;
// TODO: we use MetadataTypedColumnsetSerDe for now, till DynamicSerDe is
// ready
@@ -115,10 +138,10 @@ protected void initializeOp(Configuration hconf) throws HiveException {
distinctColIndices = conf.getDistinctColumnIndices();
numDistinctExprs = distinctColIndices.size();
- valueEval = new ExprNodeEvaluator[conf.getValueCols().size()];
+ outputValueEval = new ExprNodeEvaluator[conf.getValueCols().size()];
i = 0;
for (ExprNodeDesc e : conf.getValueCols()) {
- valueEval[i++] = ExprNodeEvaluatorFactory.get(e);
+ outputValueEval[i++] = ExprNodeEvaluatorFactory.get(e);
}
partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()];
@@ -128,6 +151,38 @@ protected void initializeOp(Configuration hconf) throws HiveException {
partitionEval[i++] = index < 0 ? ExprNodeEvaluatorFactory.get(e): keyEval[index];
}
+ if (conf.getBucketCols() != null && !conf.getBucketCols().isEmpty()) {
+ bucketEval = new ExprNodeEvaluator[conf.getBucketCols().size()];
+
+ i = 0;
+ for (ExprNodeDesc e : conf.getBucketCols()) {
+ int index = ExprNodeDescUtils.indexOf(e, keys);
+ bucketEval[i++] = index < 0 ? ExprNodeEvaluatorFactory.get(e) : keyEval[index];
+ }
+
+ // input value will not have bucket column
+ inputValueEval = new ExprNodeEvaluator[outputValueEval.length - 1];
+ System.arraycopy(outputValueEval, 0, inputValueEval, 0, outputValueEval.length - 1);
+
+ List inputColNames = conf.getOutputValueColumnNames().subList(0,
+ conf.getOutputValueColumnNames().size() - 1);
+
+ // prepare OI for output row/value
+ objectInspectors = new ArrayList(outputValueEval.length);
+ valueObjectInspector = initEvaluatorsAndReturnStruct(inputValueEval, inputColNames,
+ inputObjInspectors[0]);
+ List outColNames = conf.getOutputValueColumnNames();
+ for (ExprNodeEvaluator> ev : inputValueEval) {
+ objectInspectors.add(ev.getOutputOI());
+ }
+ objectInspectors.add(ObjectInspectorFactory.getReflectionObjectInspector(Integer.class,
+ ObjectInspectorOptions.JAVA));
+ outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector(outColNames,
+ objectInspectors);
+ buckColIdxInKey = ExprNodeDescUtils.indexOf(conf.getBucketCols().get(0), conf.getKeyCols());
+ valueObjectInspector = outputObjInspector;
+ }
+
tag = conf.getTag();
tagByte[0] = (byte) tag;
LOG.info("Using tag = " + tag);
@@ -163,6 +218,8 @@ protected void initializeOp(Configuration hconf) throws HiveException {
protected transient ObjectInspector keyObjectInspector;
protected transient ObjectInspector valueObjectInspector;
transient ObjectInspector[] partitionObjectInspectors;
+ transient ObjectInspector[] bucketObjectInspectors = null;
+ transient int buckColIdxInKey;
protected transient Object[] cachedValues;
protected transient List> distinctColIndices;
@@ -241,17 +298,35 @@ public void processOp(Object row, int tag) throws HiveException {
keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval,
distinctColIndices,
conf.getOutputKeyColumnNames(), numDistributionKeys, rowInspector);
- valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, conf
- .getOutputValueColumnNames(), rowInspector);
+ if (bucketEval == null) {
+ valueObjectInspector = initEvaluatorsAndReturnStruct(outputValueEval,
+ conf.getOutputValueColumnNames(), rowInspector);
+ } else {
+ bucketObjectInspectors = initEvaluators(bucketEval, rowInspector);
+ rowInspector = outputObjInspector;
+ }
partitionObjectInspectors = initEvaluators(partitionEval, rowInspector);
int numKeys = numDistinctExprs > 0 ? numDistinctExprs : 1;
int keyLen = numDistinctExprs > 0 ? numDistributionKeys + 1 : numDistributionKeys;
cachedKeys = new Object[numKeys][keyLen];
- cachedValues = new Object[valueEval.length];
+ cachedValues = new Object[outputValueEval.length];
}
// Determine distKeyLength (w/o distincts), and then add the first if present.
populateCachedDistributionKeys(row, 0);
+
+ // replace bucketing columns with hashcode % numBuckets
+ int buckNum = 0;
+ if (bucketEval != null) {
+ buckNum = computeBucketNumber(row, conf.getNumBuckets());
+ cachedKeys[0][buckColIdxInKey] = getLazyObject(buckNum, bucketEval[0].getOutputOI());
+
+ // in case if there are more bucketing cols, set their values in key to -1
+ for (int i = 1; i < bucketEval.length; i++) {
+ cachedKeys[0][buckColIdxInKey + i] = getLazyObject(-1, bucketEval[i].getOutputOI());
+ }
+ }
+
HiveKey firstKey = toHiveKey(cachedKeys[0], tag, null);
int distKeyLength = firstKey.getDistKeyLength();
if (numDistinctExprs > 0) {
@@ -263,8 +338,16 @@ public void processOp(Object row, int tag) throws HiveException {
int firstIndex = reducerHash.tryStoreKey(firstKey);
if (firstIndex == TopNHash.EXCLUDE) return; // Nothing to do.
// Compute value and hashcode - we'd either store or forward them.
- BytesWritable value = makeValueWritable(row);
- int hashCode = computeHashCode(row);
+ BytesWritable value;
+ int hashCode;
+ if (bucketEval == null) {
+ value = makeValueWritable(row);
+ hashCode = computeHashCode(row);
+ } else {
+ value = makeValueWritable(row, buckNum);
+ hashCode = computeHashCode(row, buckNum);
+ }
+
if (firstIndex == TopNHash.FORWARD) {
firstKey.setHashCode(hashCode);
collect(firstKey, value);
@@ -288,6 +371,38 @@ public void processOp(Object row, int tag) throws HiveException {
}
}
+ private LazyObject extends ObjectInspector> getLazyObject(int buckNum, ObjectInspector oi) {
+ byte[] ba = stringToBytesASCII(String.valueOf(buckNum));
+ LazyObject extends ObjectInspector> lo = LazyFactory.createLazyObject(oi);
+ ByteArrayRef b = new ByteArrayRef();
+ b.setData(ba);
+ lo.init(b, 0, ba.length);
+ return lo;
+ }
+
+ public static byte[] stringToBytesASCII(String str) {
+ char[] buffer = str.toCharArray();
+ byte[] b = new byte[buffer.length];
+ for (int i = 0; i < b.length; i++) {
+ b[i] = (byte) buffer[i];
+ }
+ return b;
+ }
+
+ private int computeBucketNumber(Object row, int numBuckets) throws HiveException {
+ int buckNum = 0;
+ for (int i = 0; i < bucketEval.length; i++) {
+ Object o = bucketEval[i].evaluate(row);
+ buckNum = buckNum * 31 + ObjectInspectorUtils.hashCode(o, bucketObjectInspectors[i]);
+ }
+
+ if (buckNum < 0) {
+ buckNum = -1 * buckNum;
+ }
+
+ return buckNum % numBuckets;
+ }
+
private void populateCachedDistributionKeys(Object row, int index) throws HiveException {
for (int i = 0; i < numDistributionKeys; i++) {
cachedKeys[index][i] = keyEval[i].evaluate(row);
@@ -336,6 +451,33 @@ private int computeHashCode(Object row) throws HiveException {
return keyHashCode;
}
+ private int computeHashCode(Object row, int buckNum) throws HiveException {
+ // Evaluate the HashCode
+ int keyHashCode = 0;
+ if (partitionEval.length == 0) {
+ // If no partition cols, just distribute the data uniformly to provide better
+ // load balance. If the requirement is to have a single reducer, we should set
+ // the number of reducers to 1.
+ // Use a constant seed to make the code deterministic.
+ if (random == null) {
+ random = new Random(12345);
+ }
+ keyHashCode = random.nextInt();
+ } else {
+ // partitionEval will include all columns from distribution columns i.e;
+ // partition columns + bucket columns. Hence do not include the bucket
+ // column value in hashcode computation, instead bucket number will be used
+ for (int i = 0; i < partitionEval.length - conf.getBucketCols().size(); i++) {
+ Object o = partitionEval[i].evaluate(row);
+ keyHashCode = keyHashCode * 31
+ + ObjectInspectorUtils.hashCode(o, partitionObjectInspectors[i]);
+ }
+
+ keyHashCode = keyHashCode * 31 + buckNum;
+ }
+ return keyHashCode;
+ }
+
// Serialize the keys and append the tag
protected HiveKey toHiveKey(Object obj, int tag, Integer distLength) throws SerDeException {
BinaryComparable key = (BinaryComparable)keySerializer.serialize(obj, keyObjectInspector);
@@ -367,13 +509,26 @@ protected void collect(BytesWritable keyWritable, Writable valueWritable) throws
private BytesWritable makeValueWritable(Object row) throws Exception {
// Evaluate the value
- for (int i = 0; i < valueEval.length; i++) {
- cachedValues[i] = valueEval[i].evaluate(row);
+ for (int i = 0; i < outputValueEval.length; i++) {
+ cachedValues[i] = outputValueEval[i].evaluate(row);
}
// Serialize the value
return (BytesWritable) valueSerializer.serialize(cachedValues, valueObjectInspector);
}
+ private BytesWritable makeValueWritable(Object row, int buckNum) throws Exception {
+ // Evaluate the value
+ int i;
+ for (i = 0; i < inputValueEval.length; i++) {
+ cachedValues[i] = inputValueEval[i].evaluate(row);
+ }
+
+ cachedValues[i] = buckNum;
+
+ // Serialize the value
+ return (BytesWritable) valueSerializer.serialize(cachedValues, valueObjectInspector);
+ }
+
@Override
protected void closeOp(boolean abort) throws HiveException {
if (!abort) {
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index 6df4b3f..0c6496c 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -112,6 +112,11 @@ public void initialize(HiveConf hiveConf) {
if (HiveConf.getFloatVar(hiveConf, HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE) > 0) {
transformations.add(new LimitPushdownOptimizer());
}
+ if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.DYNAMICPARTITIONING) &&
+ HiveConf.getVar(hiveConf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equals("nonstrict") &&
+ HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_SORT_DYNPART)) {
+ transformations.add(new SortedDynPartitionOptimizer());
+ }
if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES)) {
transformations.add(new StatsOptimizer());
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
new file mode 100644
index 0000000..8f326b0
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -0,0 +1,670 @@
+/**
+ * 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.optimizer;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.ExtractOperator;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
+import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
+import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.OpParseContext;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.RowResolver;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExtractDesc;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * When dynamic partitioning (with or without bucketing and sorting) is enabled,
+ * this optimization sorts the records on partition, bucket and sort columns
+ * respectively before inserting records into the destination table. This enables
+ * reducers to keep only one record writer all the time thereby reducing the
+ * the memory pressure on the reducers. This optimization will force a reducer
+ * even when hive.enforce.bucketing and hive.enforce.sorting is set to false.
+ *
+ */
+public class SortedDynPartitionOptimizer implements Transform {
+
+ @Override
+ public ParseContext transform(ParseContext pCtx) throws SemanticException {
+
+ // create a walker which walks the tree in a DFS manner while maintaining the
+ // operator stack. The dispatcher generates the plan from the operator tree
+ Map opRules = new LinkedHashMap();
+
+ String TS = TableScanOperator.getOperatorName() + "%";
+ String RS = ReduceSinkOperator.getOperatorName() + "%";
+ String FS = FileSinkOperator.getOperatorName() + "%";
+ String FIL = FilterOperator.getOperatorName() + "%";
+ String SEL = SelectOperator.getOperatorName() + "%";
+ String EX = ExtractOperator.getOperatorName() + "%";
+
+ // MR plan
+ opRules.put(new RuleRegExp("R3", RS + EX + FS), getMapReduceProc(pCtx));
+
+ // Map only plan
+ opRules.put(new RuleRegExp("R1", TS + FIL + SEL + FS), getMapOnlyProc(pCtx));
+ opRules.put(new RuleRegExp("R2", TS + SEL + FS), getMapOnlyProc(pCtx));
+
+ Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+ GraphWalker ogw = new DefaultGraphWalker(disp);
+
+ ArrayList topNodes = new ArrayList();
+ topNodes.addAll(pCtx.getTopOps().values());
+ ogw.startWalking(topNodes, null);
+
+ return pCtx;
+ }
+
+ private NodeProcessor getMapOnlyProc(ParseContext pCtx) {
+ return new SortedDynPartMapOnlyProc(pCtx);
+ }
+
+ private NodeProcessor getMapReduceProc(ParseContext pCtx) {
+ return new SortedDynPartMRProc(pCtx);
+ }
+
+ class SortedDynPartMapOnlyProc implements NodeProcessor {
+
+ protected ParseContext parseCtx;
+
+ public SortedDynPartMapOnlyProc(ParseContext pCtx) {
+ this.parseCtx = pCtx;
+ }
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+
+ // If the reduce sink has not been introduced due to bucketing/sorting, ignore it
+ FileSinkOperator fsOp = (FileSinkOperator) nd;
+
+ // skip if parent is not SEL
+ if (!(fsOp.getParentOperators().get(0) instanceof SelectOperator)) {
+ return null;
+ }
+
+ SelectOperator selOp = (SelectOperator) fsOp.getParentOperators().get(0);
+ DynamicPartitionCtx dpCtx = fsOp.getConf().getDynPartCtx();
+ Table destTable = parseCtx.getFsopToTable().get(fsOp);
+ if (destTable == null) {
+ return null;
+ }
+
+ // Get the positions for partition, bucket and sort columns in case of any
+ List bucketPositions = getBucketPositions(destTable.getBucketCols(),
+ destTable.getCols());
+ ObjectPair, List> sortOrderPositions = getSortPositionsOrder(
+ destTable.getSortCols(), destTable.getCols());
+ List sortPositions = sortOrderPositions.getFirst();
+ List sortOrder = sortOrderPositions.getSecond();
+ List partitionPositions = getPartitionPositionsColNames(dpCtx, selOp.getConf()
+ .getOutputColumnNames());
+ int numBuckets = destTable.getNumBuckets();
+ ArrayList bucketColumns = getBucketPositionsToExprNodes(bucketPositions, selOp
+ .getConf().getColList());
+
+ // update file sink descriptor
+ fsOp.getConf().setBucketCols(bucketColumns);
+ fsOp.getConf().setMultiFileSpray(false);
+ fsOp.getConf().setNumFiles(1);
+ fsOp.getConf().setTotalFiles(1);
+
+ // Insert RS and EX between SEL and FS. New order will become
+ // SEL -> RS -> EX -> FS.
+
+ // Clear off SEL children
+ selOp.getChildOperators().clear();
+
+ // Prepare ReduceSinkDesc
+ RowResolver inputRR = parseCtx.getOpParseCtx().get(selOp).getRowResolver();
+ ObjectPair pair = copyRowResolver(inputRR);
+ RowResolver outRR = pair.getSecond();
+ ArrayList valColInfo = Lists.newArrayList(selOp.getSchema().getSignature());
+ ArrayList newValueCols = Lists.newArrayList();
+ Map colExprMap = Maps.newHashMap();
+ for (ColumnInfo ci : valColInfo) {
+ newValueCols.add(new ExprNodeColumnDesc(ci.getType(), ci.getInternalName(), ci
+ .getTabAlias(), ci.isHiddenVirtualCol()));
+ colExprMap.put(ci.getInternalName(), newValueCols.get(newValueCols.size() - 1));
+ }
+ ReduceSinkDesc rsConf = getReduceSinkDesc(partitionPositions, bucketPositions, sortPositions,
+ sortOrder, newValueCols, bucketColumns, numBuckets, selOp);
+
+ // Create ReduceSink operator
+ ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap(
+ OperatorFactory.getAndMakeChild(rsConf, new RowSchema(outRR.getColumnInfos()), selOp),
+ outRR, parseCtx);
+ rsOp.setColumnExprMap(colExprMap);
+
+ // Create ExtractDesc
+ ObjectPair exPair = copyRowResolver(outRR);
+ RowResolver exRR = exPair.getSecond();
+ ExtractDesc exConf = new ExtractDesc(new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo,
+ Utilities.ReduceField.VALUE.toString(), "", false));
+
+ // Create Extract Operator
+ ExtractOperator exOp = (ExtractOperator) putOpInsertMap(
+ OperatorFactory.getAndMakeChild(exConf, new RowSchema(exRR.getColumnInfos()), rsOp),
+ exRR, parseCtx);
+
+ // Attach the newly created operators to FileSink
+ FileSinkOperator newFsOp = null;
+ try {
+ RowResolver fsRR = parseCtx.getOpParseCtx().get(fsOp).getRowResolver();
+ Table table = parseCtx.getFsopToTable().get(fsOp);
+ newFsOp = (FileSinkOperator) fsOp.cloneOp();
+ parseCtx.removeOpParseCtx(fsOp);
+ parseCtx.getFsopToTable().remove(fsOp);
+ parseCtx.getFsopToTable().put(newFsOp, table);
+ parseCtx.getOpParseCtx().put(newFsOp, new OpParseContext(fsRR));
+ newFsOp.getParentOperators().add(exOp);
+ } catch (CloneNotSupportedException e) {
+ // do not proceed with this optimization
+ e.printStackTrace();
+ return null;
+ }
+ exOp.getChildOperators().add(newFsOp);
+
+ // Set if partition sorted or partition bucket sorted
+ newFsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_SORTED);
+ if (bucketColumns.size() > 1) {
+ newFsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_BUCKET_SORTED);
+ updateRowResolvers(parseCtx, rsOp);
+ updateReduceSinkKeyTable(rsOp);
+ updateReduceSinkValueTable(rsOp);
+ }
+ return null;
+ }
+
+ }
+
+ class SortedDynPartMRProc implements NodeProcessor {
+
+ protected ParseContext parseCtx;
+
+ public SortedDynPartMRProc(ParseContext pCtx) {
+ this.parseCtx = pCtx;
+ }
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+
+ FileSinkOperator fsOp = (FileSinkOperator) nd;
+ if (!(fsOp.getParentOperators().get(0) instanceof ExtractOperator)) {
+ return null;
+ }
+ ExtractOperator exOp = (ExtractOperator) fsOp.getParentOperators().get(0);
+ if (!(exOp.getParentOperators().get(0) instanceof ReduceSinkOperator)) {
+ return null;
+ }
+ ReduceSinkOperator rsOp = (ReduceSinkOperator) exOp.getParentOperators().get(0);
+ DynamicPartitionCtx dpCtx = fsOp.getConf().getDynPartCtx();
+ HashMap> tops = parseCtx.getTopOps();
+
+ // We do not deal with multiple TS ops at this point
+ if (tops.size() > 1) {
+ return null;
+ }
+
+ TableScanOperator tsOp = null;
+ for (Map.Entry> entry : tops.entrySet()) {
+ tsOp = (TableScanOperator) entry.getValue();
+ }
+
+ Table srcTable = parseCtx.getTopToTable().get(tsOp);
+ Table destTable = parseCtx.getFsopToTable().get(fsOp);
+ if (destTable == null || srcTable == null) {
+ return null;
+ }
+
+ // Get the positions for partition, bucket and sort columns
+ List bucketPositions = getBucketPositions(destTable.getBucketCols(),
+ destTable.getCols());
+ ObjectPair, List> sortOrderPositions = getSortPositionsOrder(
+ destTable.getSortCols(), destTable.getCols());
+ List sortPositions = sortOrderPositions.getFirst();
+ List sortOrder = sortOrderPositions.getSecond();
+ List partitionPositions = getPartitionPositions(dpCtx, rsOp.getConf().getValueCols());
+ int numBuckets = destTable.getNumBuckets();
+ ArrayList bucketColumns = getBucketPositionsToExprNodes(bucketPositions, rsOp
+ .getConf().getValueCols());
+
+ // update file sink descriptor
+ fsOp.getConf().setBucketCols(bucketColumns);
+ fsOp.getConf().setMultiFileSpray(false);
+ fsOp.getConf().setNumFiles(1);
+ fsOp.getConf().setTotalFiles(1);
+
+ // update reduce sink descriptors
+ rsOp.getConf().setBucketCols(bucketColumns);
+ rsOp.getConf().setNumBuckets(numBuckets);
+ updateReduceSinkKey(partitionPositions, bucketPositions, sortPositions, sortOrder, rsOp);
+ if (bucketColumns.size() > 0) {
+ fsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_BUCKET_SORTED);
+ updateRowResolvers(parseCtx, rsOp);
+ updateReduceSinkKeyTable(rsOp);
+ updateReduceSinkValueTable(rsOp);
+ }
+ return null;
+ }
+ }
+
+ // Get the bucket positions for the table
+ private List getBucketPositions(List tabBucketCols, List tabCols) {
+ List posns = new ArrayList();
+ for (String bucketCol : tabBucketCols) {
+ int pos = 0;
+ for (FieldSchema tabCol : tabCols) {
+ if (bucketCol.equals(tabCol.getName())) {
+ posns.add(pos);
+ break;
+ }
+ pos++;
+ }
+ }
+ return posns;
+ }
+
+ public ReduceSinkDesc getReduceSinkDesc(List partitionPositions,
+ List bucketPositions, List sortPositions, List sortOrder,
+ ArrayList newValueCols, ArrayList bucketColumns, int numBuckets,
+ SelectOperator selOp) {
+
+ // Order of KEY columns
+ // 1) Partition columns
+ // 2) Bucket columns
+ // 3) Sort columns
+ List keyColsPosInVal = Lists.newArrayList();
+ ArrayList newKeyCols = Lists.newArrayList();
+ List newSortOrder = Lists.newArrayList();
+
+ keyColsPosInVal.addAll(partitionPositions);
+ keyColsPosInVal.addAll(bucketPositions);
+ keyColsPosInVal.addAll(sortPositions);
+
+ // by default partition and bucket columns are sorted in ascending order
+ Integer order = null;
+ if (sortOrder != null && !sortOrder.isEmpty()) {
+ if (sortOrder.get(0).intValue() == 1) {
+ order = 1;
+ } else {
+ order = 0;
+ }
+ }
+ for (int i = 0; i < (partitionPositions.size() + bucketPositions.size()); i++) {
+ newSortOrder.add(order);
+ }
+ newSortOrder.addAll(sortOrder);
+
+ List partColsPosInVal = Lists.newArrayList();
+ ArrayList newPartCols = Lists.newArrayList();
+
+ // Order of DISTRIBUTION columns
+ // 1) Partition columns
+ // 2) Bucket columns
+ partColsPosInVal.addAll(partitionPositions);
+ partColsPosInVal.addAll(bucketPositions);
+
+ // we will clone here as RS will update bucket column key with its
+ // corresponding with bucker number and hence their OIs
+ for (Integer idx : keyColsPosInVal) {
+ newKeyCols.add(newValueCols.get(idx).clone());
+ }
+
+ for (Integer idx : partColsPosInVal) {
+ newPartCols.add(newValueCols.get(idx).clone());
+ }
+
+ String orderStr = "";
+ for (int i = 0; i < newKeyCols.size(); i++) {
+ orderStr += "+";
+ }
+
+ // Create Key/Value TableDesc. When the operator plan is split into MR tasks,
+ // the reduce operator will initialize Extract operator with information
+ // from Key and Value TableDesc
+ List fields = PlanUtils.getFieldSchemasFromColumnList(newKeyCols, "reducesinkkey");
+ TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, orderStr);
+ ArrayList outputKeyCols = Lists.newArrayList();
+ for (int i = 0; i < newKeyCols.size(); i++) {
+ outputKeyCols.add("reducesinkkey" + i);
+ }
+
+ ArrayList outValColNames = Lists.newArrayList(selOp.getConf().getOutputColumnNames());
+ List valFields = PlanUtils.getFieldSchemasFromColumnList(newValueCols,
+ outValColNames, 0, "");
+ TableDesc valueTable = PlanUtils.getReduceValueTableDesc(valFields);
+
+ List> distinctColumnIndices = Lists.newArrayList();
+ ReduceSinkDesc rsConf = new ReduceSinkDesc(newKeyCols, newKeyCols.size(), newValueCols,
+ outputKeyCols, distinctColumnIndices, outValColNames, -1, newPartCols, -1, keyTable,
+ valueTable);
+ rsConf.setBucketCols(bucketColumns);
+ rsConf.setNumBuckets(numBuckets);
+
+ return rsConf;
+ }
+
+ /**
+ * Returns the position of partitions columns given their ExprNodeDesc
+ * @param dpCtx
+ * @param cols
+ * @return
+ */
+ private List getPartitionPositions(DynamicPartitionCtx dpCtx,
+ ArrayList cols) {
+ List posns = Lists.newArrayList();
+ List internalCols = Lists.newArrayList();
+ for (String dpCol : dpCtx.getDPColNames()) {
+ for (Map.Entry entry : dpCtx.getInputToDPCols().entrySet()) {
+ if (entry.getValue().equalsIgnoreCase(dpCol)) {
+ internalCols.add(entry.getKey());
+ }
+ }
+ }
+
+ for (String partCol : internalCols) {
+ int pos = 0;
+ for (ExprNodeDesc tabCol : cols) {
+ String colName = ((ExprNodeColumnDesc) tabCol).getColumn();
+ if (partCol.equals(colName)) {
+ posns.add(pos);
+ break;
+ }
+ pos++;
+ }
+ }
+ return posns;
+ }
+
+ /**
+ * Returns the position of partition columns given their Column Names
+ * @param dpCtx
+ * @param colNames
+ * @return
+ */
+ private List getPartitionPositionsColNames(DynamicPartitionCtx dpCtx,
+ List colNames) {
+ List posns = Lists.newArrayList();
+ List internalCols = Lists.newArrayList();
+ for (String dpCol : dpCtx.getDPColNames()) {
+ for (Map.Entry entry : dpCtx.getInputToDPCols().entrySet()) {
+ if (entry.getValue().equalsIgnoreCase(dpCol)) {
+ internalCols.add(entry.getKey());
+ }
+ }
+ }
+
+ for (String partCol : internalCols) {
+ int pos = 0;
+ for (String tabCol : colNames) {
+ if (partCol.equals(tabCol)) {
+ posns.add(pos);
+ break;
+ }
+ pos++;
+ }
+ }
+ return posns;
+ }
+
+ /**
+ * Get the sort positions and sort order for the sort columns
+ * @param tabSortCols
+ * @param tabCols
+ * @return
+ */
+ private ObjectPair, List> getSortPositionsOrder(List tabSortCols,
+ List tabCols) {
+ List sortPositions = Lists.newArrayList();
+ List sortOrders = Lists.newArrayList();
+ for (Order sortCol : tabSortCols) {
+ int pos = 0;
+ for (FieldSchema tabCol : tabCols) {
+ if (sortCol.getCol().equals(tabCol.getName())) {
+ sortPositions.add(pos);
+ sortOrders.add(sortCol.getOrder());
+ break;
+ }
+ pos++;
+ }
+ }
+ return new ObjectPair, List>(sortPositions, sortOrders);
+ }
+
+ /**
+ * Refreshes the Key TableDesc based on updates to ReduceSinkDesc
+ * @param rsOp
+ */
+ private void updateReduceSinkKeyTable(ReduceSinkOperator rsOp) {
+ ArrayList keyCols = rsOp.getConf().getKeyCols();
+ String order = rsOp.getConf().getOrder();
+ String newOrder = "";
+ if (order == null || order.isEmpty() || order.startsWith("+")) {
+ order = "+";
+ } else {
+ order = "-";
+ }
+
+ for (int i = 0; i < keyCols.size(); i++) {
+ newOrder += order;
+ }
+ List fields = PlanUtils.getFieldSchemasFromColumnList(keyCols, "reducesinkkey");
+ TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, newOrder);
+ ArrayList outputKeyCols = Lists.newArrayList();
+ for (int i = 0; i < keyCols.size(); i++) {
+ outputKeyCols.add("reducesinkkey" + i);
+ }
+ rsOp.getConf().setKeySerializeInfo(keyTable);
+ rsOp.getConf().setOutputKeyColumnNames(outputKeyCols);
+ }
+
+ /**
+ * Refreshes the Value TableDesc based on updates to ReduceSinkDesc
+ * @param rsOp
+ */
+ private void updateReduceSinkValueTable(ReduceSinkOperator rsOp) {
+ ArrayList valueCols = rsOp.getConf().getValueCols();
+ ArrayList outValColNames = rsOp.getConf().getOutputValueColumnNames();
+ List fields = PlanUtils.getFieldSchemasFromColumnList(valueCols, outValColNames,
+ 0, "");
+ TableDesc valueTable = PlanUtils.getReduceValueTableDesc(fields);
+ rsOp.getConf().setValueSerializeInfo(valueTable);
+ }
+
+ private ArrayList getBucketPositionsToExprNodes(List bucketPositions,
+ List valueCols) {
+ ArrayList bucketCols = Lists.newArrayList();
+
+ for (Integer idx : bucketPositions) {
+ bucketCols.add(valueCols.get(idx));
+ }
+
+ return bucketCols;
+ }
+
+ private void updateReduceSinkKey(List partitionPositions, List bucketPositions,
+ List sortPositions, List sortOrder, ReduceSinkOperator rsOp) {
+
+ List valueCols = rsOp.getConf().getValueCols();
+
+ // Order of KEY columns
+ // 1) Partition columns
+ // 2) Bucket columns
+ // 3) Sort columns
+ List keyColsPosInVal = Lists.newArrayList();
+ ArrayList newKeyCols = Lists.newArrayList();
+ List newSortOrder = Lists.newArrayList();
+
+ keyColsPosInVal.addAll(partitionPositions);
+ keyColsPosInVal.addAll(bucketPositions);
+ keyColsPosInVal.addAll(sortPositions);
+
+ // by default partition and bucket columns are sorted in ascending order
+ Integer order = null;
+ if (sortOrder != null && !sortOrder.isEmpty()) {
+ if (sortOrder.get(0).intValue() == 1) {
+ order = 1;
+ } else {
+ order = 0;
+ }
+ }
+ for (int i = 0; i < (partitionPositions.size() + bucketPositions.size()); i++) {
+ newSortOrder.add(order);
+ }
+ newSortOrder.addAll(sortOrder);
+
+ List partColsPosInVal = Lists.newArrayList();
+ ArrayList newPartCols = Lists.newArrayList();
+
+ // Order of DISTRIBUTION columns
+ // 1) Partition columns
+ // 2) Bucket columns
+ partColsPosInVal.addAll(partitionPositions);
+ partColsPosInVal.addAll(bucketPositions);
+
+ // we will clone here as RS will update bucket column with bucker number
+ for (Integer idx : keyColsPosInVal) {
+ newKeyCols.add(valueCols.get(idx).clone());
+ }
+
+ for (Integer idx : partColsPosInVal) {
+ newPartCols.add(valueCols.get(idx).clone());
+ }
+
+ rsOp.getConf().setKeyCols(newKeyCols);
+ rsOp.getConf().setNumDistributionKeys(newKeyCols.size());
+ rsOp.getConf().setPartitionCols(newPartCols);
+ }
+
+ /**
+ * Updated RowResolvers of RS and all of its children
+ * @param pCtx
+ * @param rsOp
+ */
+ private void updateRowResolvers(ParseContext pCtx, ReduceSinkOperator rsOp) {
+ ArrayList valueCols = rsOp.getConf().getValueCols();
+ Map colExprMap = rsOp.getColumnExprMap();
+ RowResolver inputRR = pCtx.getOpParseCtx().get(rsOp).getRowResolver();
+ ArrayList outColInfos = inputRR.getColumnInfos();
+ ObjectPair tabToRR = copyRowResolver(inputRR);
+ String tabAlias = tabToRR.getFirst();
+ RowResolver outRR = tabToRR.getSecond();
+
+ // insert new hidden column _bcol0 to value columns
+ ColumnInfo buckCol = new ColumnInfo("_bcol0", TypeInfoFactory.intTypeInfo, tabAlias, true);
+ ExprNodeColumnDesc encd = new ExprNodeColumnDesc(buckCol.getType(), buckCol.getInternalName(),
+ buckCol.getAlias(), buckCol.getIsVirtualCol());
+ valueCols.add(encd);
+ colExprMap.put(buckCol.getInternalName(), valueCols.get(valueCols.size() - 1));
+ outColInfos.add(buckCol);
+ outRR.put(tabAlias, "_bcol0", buckCol);
+ rsOp.getConf().getOutputValueColumnNames().add("_bcol0");
+ rsOp.setSchema(outRR.getRowSchema());
+ pCtx.getOpParseCtx().get(rsOp).setRowResolver(outRR);
+
+ // update the row resolver of children
+ for (Operator extends OperatorDesc> child : rsOp.getChildOperators()) {
+ updateChildRowResolvers(pCtx, outRR, child);
+ }
+ }
+
+ private void updateChildRowResolvers(ParseContext pCtx, RowResolver inputRR,
+ Operator extends OperatorDesc> op) {
+ if (op == null) {
+ return;
+ }
+
+ ObjectPair tabToRR = copyRowResolver(inputRR);
+ RowResolver outRR = tabToRR.getSecond();
+
+ op.setSchema(outRR.getRowSchema());
+ pCtx.getOpParseCtx().get(op).setRowResolver(outRR);
+
+ for (Operator extends OperatorDesc> child : op.getChildOperators()) {
+ updateChildRowResolvers(pCtx, outRR, child);
+ }
+ }
+
+ private Operator extends Serializable> putOpInsertMap(Operator> op, RowResolver rr,
+ ParseContext context) {
+ OpParseContext ctx = new OpParseContext(rr);
+ context.getOpParseCtx().put(op, ctx);
+ return op;
+ }
+
+ private ObjectPair copyRowResolver(RowResolver inputRR) {
+ ObjectPair output = new ObjectPair();
+ RowResolver outRR = new RowResolver();
+ int pos = 0;
+ String tabAlias = null;
+
+ for (ColumnInfo colInfo : inputRR.getColumnInfos()) {
+ String[] info = inputRR.reverseLookup(colInfo.getInternalName());
+ tabAlias = info[0];
+ outRR.put(info[0], info[1], new ColumnInfo(SemanticAnalyzer.getColumnInternalName(pos),
+ colInfo.getType(), info[0], colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()));
+ pos++;
+ }
+ output.setFirst(tabAlias);
+ output.setSecond(outRR);
+ return output;
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index 51ebea4..8020d6d 100644
--- ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -30,6 +30,12 @@
@Explain(displayName = "File Output Operator")
public class FileSinkDesc extends AbstractOperatorDesc {
private static final long serialVersionUID = 1L;
+
+ public enum DPSortState {
+ NONE, PARTITION_SORTED, PARTITION_BUCKET_SORTED
+ }
+
+ private DPSortState dpSortState;
private Path dirName;
// normally statsKeyPref will be the same as dirName, but the latter
// could be changed in local execution optimization
@@ -45,6 +51,7 @@
private boolean canBeMerged;
private int totalFiles;
private ArrayList partitionCols;
+ private ArrayList bucketCols;
private int numFiles;
private DynamicPartitionCtx dpCtx;
private String staticSpec; // static partition spec ends with a '/'
@@ -96,6 +103,8 @@ public FileSinkDesc(final Path dirName, final TableDesc tableInfo,
this.totalFiles = totalFiles;
this.partitionCols = partitionCols;
this.dpCtx = dpCtx;
+ this.bucketCols = null;
+ this.dpSortState = DPSortState.NONE;
}
public FileSinkDesc(final Path dirName, final TableDesc tableInfo,
@@ -110,6 +119,8 @@ public FileSinkDesc(final Path dirName, final TableDesc tableInfo,
this.numFiles = 1;
this.totalFiles = 1;
this.partitionCols = null;
+ this.bucketCols = null;
+ this.dpSortState = DPSortState.NONE;
}
@Override
@@ -128,6 +139,8 @@ public Object clone() throws CloneNotSupportedException {
ret.setStatsReliable(statsReliable);
ret.setMaxStatsKeyPrefixLength(maxStatsKeyPrefixLength);
ret.setStatsCollectRawDataSize(statsCollectRawDataSize);
+ ret.setBucketCols(bucketCols);
+ ret.setDpSortState(dpSortState);
return (Object) ret;
}
@@ -381,4 +394,20 @@ public boolean isRemovedReduceSinkBucketSort() {
public void setRemovedReduceSinkBucketSort(boolean removedReduceSinkBucketSort) {
this.removedReduceSinkBucketSort = removedReduceSinkBucketSort;
}
+
+ public ArrayList getBucketCols() {
+ return bucketCols;
+ }
+
+ public void setBucketCols(ArrayList bucketCols) {
+ this.bucketCols = bucketCols;
+ }
+
+ public DPSortState getDpSortState() {
+ return dpSortState;
+ }
+
+ public void setDpSortState(DPSortState dpSortState) {
+ this.dpSortState = dpSortState;
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
index f88a120..fca1fd5 100644
--- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
@@ -74,6 +74,12 @@
private int numReducers;
+ /**
+ * Bucket information
+ */
+ private int numBuckets;
+ private List bucketCols;
+
private int topN = -1;
private float topNMemoryUsage = -1;
private boolean mapGroupBy; // for group-by, values with same key on top-K should be forwarded
@@ -100,6 +106,8 @@ public ReduceSinkDesc(ArrayList keyCols,
this.keySerializeInfo = keySerializeInfo;
this.valueSerializeInfo = valueSerializeInfo;
this.distinctColumnIndices = distinctColumnIndices;
+ this.setNumBuckets(-1);
+ this.setBucketCols(null);
}
@Override
@@ -122,6 +130,8 @@ public Object clone() {
desc.setPartitionCols((ArrayList) getPartitionCols().clone());
desc.setKeySerializeInfo((TableDesc) getKeySerializeInfo().clone());
desc.setValueSerializeInfo((TableDesc) getValueSerializeInfo().clone());
+ desc.setNumBuckets(numBuckets);
+ desc.setBucketCols(bucketCols);
return desc;
}
@@ -299,4 +309,20 @@ public String getOutputName() {
public void setOutputName(String outputName) {
this.outputName = outputName;
}
+
+ public int getNumBuckets() {
+ return numBuckets;
+ }
+
+ public void setNumBuckets(int numBuckets) {
+ this.numBuckets = numBuckets;
+ }
+
+ public List getBucketCols() {
+ return bucketCols;
+ }
+
+ public void setBucketCols(List bucketCols) {
+ this.bucketCols = bucketCols;
+ }
}
diff --git ql/src/test/queries/clientpositive/dynpart_sort_optimization.q ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
new file mode 100644
index 0000000..0458163
--- /dev/null
+++ ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
@@ -0,0 +1,98 @@
+set hive.optimize.sort.dynamic.partition=true;
+set hive.exec.dynamic.partition=true;
+set hive.exec.max.dynamic.partitions=1000;
+set hive.exec.max.dynamic.partitions.pernode=1000;
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.enforce.bucketing=false;
+set hive.enforce.sorting=false;
+
+create table over1k(
+ t tinyint,
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ row format delimited
+ fields terminated by '|';
+
+load data local inpath '../../data/files/over1k' into table over1k;
+
+create table over1k_part(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (ds string, t tinyint);
+
+create table over1k_part_buck(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (t tinyint)
+ clustered by (si) into 4 buckets;
+
+create table over1k_part_buck_sort(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (t tinyint)
+ clustered by (si)
+ sorted by (f) into 4 buckets;
+
+-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+
+insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting=true;
+
+-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+
+insert into table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+insert into table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+insert into table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27;
+
+desc formatted over1k_part partition(ds="foo",t=27);
+desc formatted over1k_part partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_buck partition(t=27);
+desc formatted over1k_part_buck partition(t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_buck_sort partition(t=27);
+desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select count(*) from over1k_part;
+select count(*) from over1k_part_buck;
+select count(*) from over1k_part_buck_sort;
diff --git ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
new file mode 100644
index 0000000..ef60283
--- /dev/null
+++ ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
@@ -0,0 +1,2103 @@
+PREHOOK: query: create table over1k(
+ t tinyint,
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ row format delimited
+ fields terminated by '|'
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table over1k(
+ t tinyint,
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ row format delimited
+ fields terminated by '|'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@over1k
+PREHOOK: query: load data local inpath '../../data/files/over1k' into table over1k
+PREHOOK: type: LOAD
+PREHOOK: Output: default@over1k
+POSTHOOK: query: load data local inpath '../../data/files/over1k' into table over1k
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@over1k
+PREHOOK: query: create table over1k_part(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (ds string, t tinyint)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table over1k_part(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (ds string, t tinyint)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@over1k_part
+PREHOOK: query: create table over1k_part_buck(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (t tinyint)
+ clustered by (si) into 4 buckets
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table over1k_part_buck(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (t tinyint)
+ clustered by (si) into 4 buckets
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@over1k_part_buck
+PREHOOK: query: create table over1k_part_buck_sort(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (t tinyint)
+ clustered by (si)
+ sorted by (f) into 4 buckets
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table over1k_part_buck_sort(
+ si smallint,
+ i int,
+ b bigint,
+ f float,
+ d double,
+ bo boolean,
+ s string,
+ ts timestamp,
+ dec decimal(4,2),
+ bin binary)
+ partitioned by (t tinyint)
+ clustered by (si)
+ sorted by (f) into 4 buckets
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@over1k_part_buck_sort
+PREHOOK: query: -- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+POSTHOOK: query: -- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: over1k
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (t is null or (t = 27)) (type: boolean)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary), t (type: tinyint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: tinyint)
+ sort order: +
+ Map-reduce partition columns: _col10 (type: tinyint)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: double), _col5 (type: boolean), _col6 (type: string), _col7 (type: timestamp), _col8 (type: decimal(4,2)), _col9 (type: binary), _col10 (type: tinyint)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ ds foo
+ t
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+
+PREHOOK: query: explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+POSTHOOK: query: explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: over1k
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (t is null or (t = 27)) (type: boolean)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary), t (type: tinyint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: tinyint), _col0 (type: smallint)
+ sort order: ++
+ Map-reduce partition columns: _col10 (type: tinyint), _col0 (type: smallint)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: double), _col5 (type: boolean), _col6 (type: string), _col7 (type: timestamp), _col8 (type: decimal(4,2)), _col9 (type: binary), _col10 (type: tinyint), _bcol0 (type: int)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ t
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+
+PREHOOK: query: explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+POSTHOOK: query: explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: over1k
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (t is null or (t = 27)) (type: boolean)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary), t (type: tinyint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: tinyint), _col0 (type: smallint), _col3 (type: float)
+ sort order: +++
+ Map-reduce partition columns: _col10 (type: tinyint), _col0 (type: smallint)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: double), _col5 (type: boolean), _col6 (type: string), _col7 (type: timestamp), _col8 (type: decimal(4,2)), _col9 (type: binary), _col10 (type: tinyint), _bcol0 (type: int)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck_sort
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ t
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck_sort
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+
+PREHOOK: query: insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k
+PREHOOK: Output: default@over1k_part@ds=foo
+POSTHOOK: query: insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k
+POSTHOOK: Output: default@over1k_part@ds=foo/t=27
+POSTHOOK: Output: default@over1k_part@ds=foo/t=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k
+PREHOOK: Output: default@over1k_part_buck
+POSTHOOK: query: insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k
+POSTHOOK: Output: default@over1k_part_buck@t=27
+POSTHOOK: Output: default@over1k_part_buck@t=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k
+PREHOOK: Output: default@over1k_part_buck_sort
+POSTHOOK: query: insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k
+POSTHOOK: Output: default@over1k_part_buck_sort@t=27
+POSTHOOK: Output: default@over1k_part_buck_sort@t=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: -- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+POSTHOOK: query: -- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: over1k
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (t is null or (t = 27)) (type: boolean)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary), t (type: tinyint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: tinyint)
+ sort order: +
+ Map-reduce partition columns: _col10 (type: tinyint)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: double), _col5 (type: boolean), _col6 (type: string), _col7 (type: timestamp), _col8 (type: decimal(4,2)), _col9 (type: binary), _col10 (type: tinyint)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ ds foo
+ t
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+
+PREHOOK: query: explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+POSTHOOK: query: explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: over1k
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (t is null or (t = 27)) (type: boolean)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary), t (type: tinyint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: tinyint), _col0 (type: smallint)
+ sort order: ++
+ Map-reduce partition columns: _col10 (type: tinyint), _col0 (type: smallint)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: double), _col5 (type: boolean), _col6 (type: string), _col7 (type: timestamp), _col8 (type: decimal(4,2)), _col9 (type: binary), _col10 (type: tinyint), _bcol0 (type: int)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ t
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+
+PREHOOK: query: explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+POSTHOOK: query: explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: over1k
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (t is null or (t = 27)) (type: boolean)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary), t (type: tinyint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: tinyint), _col0 (type: smallint), _col3 (type: float)
+ sort order: +++
+ Map-reduce partition columns: _col10 (type: tinyint), _col0 (type: smallint)
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: double), _col5 (type: boolean), _col6 (type: string), _col7 (type: timestamp), _col8 (type: decimal(4,2)), _col9 (type: binary), _col10 (type: tinyint), _bcol0 (type: int)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 274 Data size: 106636 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck_sort
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ t
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.over1k_part_buck_sort
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+
+PREHOOK: query: insert into table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k
+PREHOOK: Output: default@over1k_part@ds=foo
+POSTHOOK: query: insert into table over1k_part partition(ds="foo", t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k
+POSTHOOK: Output: default@over1k_part@ds=foo/t=27
+POSTHOOK: Output: default@over1k_part@ds=foo/t=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: insert into table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k
+PREHOOK: Output: default@over1k_part_buck
+POSTHOOK: query: insert into table over1k_part_buck partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k
+POSTHOOK: Output: default@over1k_part_buck@t=27
+POSTHOOK: Output: default@over1k_part_buck@t=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: insert into table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k
+PREHOOK: Output: default@over1k_part_buck_sort
+POSTHOOK: query: insert into table over1k_part_buck_sort partition(t) select si,i,b,f,d,bo,s,ts,dec,bin,t from over1k where t is null or t=27
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k
+POSTHOOK: Output: default@over1k_part_buck_sort@t=27
+POSTHOOK: Output: default@over1k_part_buck_sort@t=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: desc formatted over1k_part partition(ds="foo",t=27)
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted over1k_part partition(ds="foo",t=27)
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+# col_name data_type comment
+
+si smallint None
+i int None
+b bigint None
+f float None
+d double None
+bo boolean None
+s string None
+ts timestamp None
+dec decimal(4,2) None
+bin binary None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+t tinyint None
+
+# Detailed Partition Information
+Partition Value: [foo, 27]
+Database: default
+Table: over1k_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ COLUMN_STATS_ACCURATE true
+ numFiles 2
+ numRows 16
+ rawDataSize 1636
+ totalSize 3304
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: desc formatted over1k_part partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__")
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted over1k_part partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__")
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+# col_name data_type comment
+
+si smallint None
+i int None
+b bigint None
+f float None
+d double None
+bo boolean None
+s string None
+ts timestamp None
+dec decimal(4,2) None
+bin binary None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+t tinyint None
+
+# Detailed Partition Information
+Partition Value: [foo, __HIVE_DEFAULT_PARTITION__]
+Database: default
+Table: over1k_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ COLUMN_STATS_ACCURATE true
+ numFiles 2
+ numRows 3
+ rawDataSize 307
+ totalSize 620
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: desc formatted over1k_part_buck partition(t=27)
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted over1k_part_buck partition(t=27)
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+# col_name data_type comment
+
+si smallint None
+i int None
+b bigint None
+f float None
+d double None
+bo boolean None
+s string None
+ts timestamp None
+dec decimal(4,2) None
+bin binary None
+
+# Partition Information
+# col_name data_type comment
+
+t tinyint None
+
+# Detailed Partition Information
+Partition Value: [27]
+Database: default
+Table: over1k_part_buck
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ COLUMN_STATS_ACCURATE true
+ numFiles 8
+ numRows 16
+ rawDataSize 1636
+ totalSize 3304
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: 4
+Bucket Columns: [si]
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: desc formatted over1k_part_buck partition(t="__HIVE_DEFAULT_PARTITION__")
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted over1k_part_buck partition(t="__HIVE_DEFAULT_PARTITION__")
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+# col_name data_type comment
+
+si smallint None
+i int None
+b bigint None
+f float None
+d double None
+bo boolean None
+s string None
+ts timestamp None
+dec decimal(4,2) None
+bin binary None
+
+# Partition Information
+# col_name data_type comment
+
+t tinyint None
+
+# Detailed Partition Information
+Partition Value: [__HIVE_DEFAULT_PARTITION__]
+Database: default
+Table: over1k_part_buck
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ COLUMN_STATS_ACCURATE true
+ numFiles 6
+ numRows 0
+ rawDataSize 0
+ totalSize 620
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: 4
+Bucket Columns: [si]
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: desc formatted over1k_part_buck_sort partition(t=27)
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted over1k_part_buck_sort partition(t=27)
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+# col_name data_type comment
+
+si smallint None
+i int None
+b bigint None
+f float None
+d double None
+bo boolean None
+s string None
+ts timestamp None
+dec decimal(4,2) None
+bin binary None
+
+# Partition Information
+# col_name data_type comment
+
+t tinyint None
+
+# Detailed Partition Information
+Partition Value: [27]
+Database: default
+Table: over1k_part_buck_sort
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ COLUMN_STATS_ACCURATE true
+ numFiles 8
+ numRows 16
+ rawDataSize 1636
+ totalSize 3304
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: 4
+Bucket Columns: [si]
+Sort Columns: [Order(col:f, order:1)]
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__")
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__")
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+# col_name data_type comment
+
+si smallint None
+i int None
+b bigint None
+f float None
+d double None
+bo boolean None
+s string None
+ts timestamp None
+dec decimal(4,2) None
+bin binary None
+
+# Partition Information
+# col_name data_type comment
+
+t tinyint None
+
+# Detailed Partition Information
+Partition Value: [__HIVE_DEFAULT_PARTITION__]
+Database: default
+Table: over1k_part_buck_sort
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ COLUMN_STATS_ACCURATE true
+ numFiles 6
+ numRows 0
+ rawDataSize 0
+ totalSize 620
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: 4
+Bucket Columns: [si]
+Sort Columns: [Order(col:f, order:1)]
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: select count(*) from over1k_part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k_part
+PREHOOK: Input: default@over1k_part@ds=foo/t=27
+PREHOOK: Input: default@over1k_part@ds=foo/t=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from over1k_part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k_part
+POSTHOOK: Input: default@over1k_part@ds=foo/t=27
+POSTHOOK: Input: default@over1k_part@ds=foo/t=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+38
+PREHOOK: query: select count(*) from over1k_part_buck
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k_part_buck
+PREHOOK: Input: default@over1k_part_buck@t=27
+PREHOOK: Input: default@over1k_part_buck@t=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from over1k_part_buck
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k_part_buck
+POSTHOOK: Input: default@over1k_part_buck@t=27
+POSTHOOK: Input: default@over1k_part_buck@t=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+38
+PREHOOK: query: select count(*) from over1k_part_buck_sort
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over1k_part_buck_sort
+PREHOOK: Input: default@over1k_part_buck_sort@t=27
+PREHOOK: Input: default@over1k_part_buck_sort@t=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from over1k_part_buck_sort
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over1k_part_buck_sort
+POSTHOOK: Input: default@over1k_part_buck_sort@t=27
+POSTHOOK: Input: default@over1k_part_buck_sort@t=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part PARTITION(ds=foo,t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=27).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(over1k)over1k.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bin SIMPLE [(over1k)over1k.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).bo SIMPLE [(over1k)over1k.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(over1k)over1k.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).dec SIMPLE [(over1k)over1k.FieldSchema(name:dec, type:decimal(4,2), comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).f SIMPLE [(over1k)over1k.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).i SIMPLE [(over1k)over1k.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).s SIMPLE [(over1k)over1k.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).si SIMPLE [(over1k)over1k.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over1k_part_buck_sort PARTITION(t=__HIVE_DEFAULT_PARTITION__).ts SIMPLE [(over1k)over1k.FieldSchema(name:ts, type:timestamp, comment:null), ]
+38
diff --git ql/src/test/results/clientpositive/insert_into6.q.out ql/src/test/results/clientpositive/insert_into6.q.out
index 07316a0..34a7194 100644
--- ql/src/test/results/clientpositive/insert_into6.q.out
+++ ql/src/test/results/clientpositive/insert_into6.q.out
@@ -134,13 +134,8 @@ POSTHOOK: Lineage: insert_into6a PARTITION(ds=2).key EXPRESSION [(src)src.FieldS
POSTHOOK: Lineage: insert_into6a PARTITION(ds=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
- Stage-4
- Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+ Stage-0 depends on stages: Stage-1
Stage-2 depends on stages: Stage-0
- Stage-3
- Stage-5
- Stage-6 depends on stages: Stage-5
STAGE PLANS:
Stage: Stage-1
@@ -153,23 +148,23 @@ STAGE PLANS:
expressions: key (type: int), value (type: string), ds (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 250 Data size: 2680 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
+ Reduce Output Operator
+ key expressions: _col2 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col2 (type: string)
Statistics: Num rows: 250 Data size: 2680 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.insert_into6b
-
- Stage: Stage-7
- Conditional Operator
-
- Stage: Stage-4
- Move Operator
- files:
- hdfs directory: true
-#### A masked pattern was here ####
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 250 Data size: 2680 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 250 Data size: 2680 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.insert_into6b
Stage: Stage-0
Move Operator
@@ -186,36 +181,6 @@ STAGE PLANS:
Stage: Stage-2
Stats-Aggr Operator
- Stage: Stage-3
- Map Reduce
- Map Operator Tree:
- TableScan
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.insert_into6b
-
- Stage: Stage-5
- Map Reduce
- Map Operator Tree:
- TableScan
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.insert_into6b
-
- Stage: Stage-6
- Move Operator
- files:
- hdfs directory: true
-#### A masked pattern was here ####
-
PREHOOK: query: INSERT INTO TABLE insert_into6b PARTITION (ds) SELECT * FROM insert_into6a
PREHOOK: type: QUERY
PREHOOK: Input: default@insert_into6a
diff --git ql/src/test/results/clientpositive/stats_empty_dyn_part.q.out ql/src/test/results/clientpositive/stats_empty_dyn_part.q.out
index a5caf2e..90cdbce 100644
--- ql/src/test/results/clientpositive/stats_empty_dyn_part.q.out
+++ ql/src/test/results/clientpositive/stats_empty_dyn_part.q.out
@@ -17,13 +17,8 @@ POSTHOOK: query: explain insert overwrite table tmptable partition (part) select
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
- Stage-4
- Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+ Stage-0 depends on stages: Stage-1
Stage-2 depends on stages: Stage-0
- Stage-3
- Stage-5
- Stage-6 depends on stages: Stage-5
STAGE PLANS:
Stage: Stage-1
@@ -39,23 +34,23 @@ STAGE PLANS:
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.tmptable
-
- Stage: Stage-7
- Conditional Operator
-
- Stage: Stage-4
- Move Operator
- files:
- hdfs directory: true
-#### A masked pattern was here ####
+ value expressions: _col0 (type: string), _col1 (type: string)
+ Reduce Operator Tree:
+ Extract
+ Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tmptable
Stage: Stage-0
Move Operator
@@ -72,36 +67,6 @@ STAGE PLANS:
Stage: Stage-2
Stats-Aggr Operator
- Stage: Stage-3
- Map Reduce
- Map Operator Tree:
- TableScan
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.tmptable
-
- Stage: Stage-5
- Map Reduce
- Map Operator Tree:
- TableScan
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.tmptable
-
- Stage: Stage-6
- Move Operator
- files:
- hdfs directory: true
-#### A masked pattern was here ####
-
PREHOOK: query: insert overwrite table tmptable partition (part) select key, value from src where key = 'no_such_value'
PREHOOK: type: QUERY
PREHOOK: Input: default@src