diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index a53fc1a..f9a7b54 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -358,6 +358,7 @@ minillaplocal.shared.query.files=alter_merge_2_orc.q,\ vector_varchar_mapjoin1.q,\ vector_varchar_simple.q,\ vector_when_case_null.q,\ + vector_windowing_navfn.q,\ vectorization_0.q,\ vectorization_1.q,\ vectorization_10.q,\ diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java index fc5aea5..b0add0a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java @@ -83,6 +83,7 @@ // This is map of which vectorized row batch columns are the key columns. // And, their types. + protected boolean isEmptyKey; protected int[] reduceSinkKeyColumnMap; protected TypeInfo[] reduceSinkKeyTypeInfos; @@ -91,6 +92,7 @@ // This is map of which vectorized row batch columns are the value columns. // And, their types. + protected boolean isEmptyValue; protected int[] reduceSinkValueColumnMap; protected TypeInfo[] reduceSinkValueTypeInfos; @@ -159,15 +161,21 @@ public VectorReduceSinkCommonOperator(CompilationOpContext ctx, vectorReduceSinkInfo = vectorDesc.getVectorReduceSinkInfo(); this.vContext = vContext; - // Since a key expression can be a calculation and the key will go into a scratch column, - // we need the mapping and type information. - reduceSinkKeyColumnMap = vectorReduceSinkInfo.getReduceSinkKeyColumnMap(); - reduceSinkKeyTypeInfos = vectorReduceSinkInfo.getReduceSinkKeyTypeInfos(); - reduceSinkKeyExpressions = vectorReduceSinkInfo.getReduceSinkKeyExpressions(); + isEmptyKey = vectorDesc.getIsEmptyKey(); + if (!isEmptyKey) { + // Since a key expression can be a calculation and the key will go into a scratch column, + // we need the mapping and type information. + reduceSinkKeyColumnMap = vectorReduceSinkInfo.getReduceSinkKeyColumnMap(); + reduceSinkKeyTypeInfos = vectorReduceSinkInfo.getReduceSinkKeyTypeInfos(); + reduceSinkKeyExpressions = vectorReduceSinkInfo.getReduceSinkKeyExpressions(); + } - reduceSinkValueColumnMap = vectorReduceSinkInfo.getReduceSinkValueColumnMap(); - reduceSinkValueTypeInfos = vectorReduceSinkInfo.getReduceSinkValueTypeInfos(); - reduceSinkValueExpressions = vectorReduceSinkInfo.getReduceSinkValueExpressions(); + isEmptyValue = vectorDesc.getIsEmptyValue(); + if (!isEmptyValue) { + reduceSinkValueColumnMap = vectorReduceSinkInfo.getReduceSinkValueColumnMap(); + reduceSinkValueTypeInfos = vectorReduceSinkInfo.getReduceSinkValueTypeInfos(); + reduceSinkValueExpressions = vectorReduceSinkInfo.getReduceSinkValueExpressions(); + } } // Get the sort order @@ -304,26 +312,33 @@ protected void initializeOp(Configuration hconf) throws HiveException { LOG.info("Using tag = " + (int) reduceTagByte); } - TableDesc keyTableDesc = conf.getKeySerializeInfo(); - boolean[] columnSortOrder = - getColumnSortOrder(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length); - byte[] columnNullMarker = - getColumnNullMarker(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length, columnSortOrder); - byte[] columnNotNullMarker = - getColumnNotNullMarker(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length, columnSortOrder); - - keyBinarySortableSerializeWrite = new BinarySortableSerializeWrite(columnSortOrder, - columnNullMarker, columnNotNullMarker); - - valueLazyBinarySerializeWrite = new LazyBinarySerializeWrite(reduceSinkValueColumnMap.length); - - valueVectorSerializeRow = - new VectorSerializeRow( - valueLazyBinarySerializeWrite); - valueVectorSerializeRow.init(reduceSinkValueTypeInfos, reduceSinkValueColumnMap); + if (!isEmptyKey) { + TableDesc keyTableDesc = conf.getKeySerializeInfo(); + boolean[] columnSortOrder = + getColumnSortOrder(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length); + byte[] columnNullMarker = + getColumnNullMarker(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length, columnSortOrder); + byte[] columnNotNullMarker = + getColumnNotNullMarker(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length, columnSortOrder); + + keyBinarySortableSerializeWrite = + new BinarySortableSerializeWrite( + columnSortOrder, + columnNullMarker, + columnNotNullMarker); + } - valueOutput = new Output(); - valueVectorSerializeRow.setOutput(valueOutput); + if (!isEmptyValue) { + valueLazyBinarySerializeWrite = new LazyBinarySerializeWrite(reduceSinkValueColumnMap.length); + + valueVectorSerializeRow = + new VectorSerializeRow( + valueLazyBinarySerializeWrite); + valueVectorSerializeRow.init(reduceSinkValueTypeInfos, reduceSinkValueColumnMap); + + valueOutput = new Output(); + valueVectorSerializeRow.setOutput(valueOutput); + } keyWritable = new HiveKey(); @@ -332,6 +347,20 @@ protected void initializeOp(Configuration hconf) throws HiveException { batchCounter = 0; } + protected void initializeEmptyKey(int tag) { + + // Use the same logic as ReduceSinkOperator.toHiveKey. + // + if (tag == -1 || reduceSkipTag) { + keyWritable.setSize(0); + } else { + keyWritable.setSize(1); + keyWritable.get()[0] = reduceTagByte; + } + keyWritable.setDistKeyLength(0); + keyWritable.setHashCode(0); + } + protected void collect(BytesWritable keyWritable, Writable valueWritable) throws IOException { // Since this is a terminal operator, update counters explicitly - // forward is not called diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkEmptyKeyOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkEmptyKeyOperator.java new file mode 100644 index 0000000..b84bfad --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkEmptyKeyOperator.java @@ -0,0 +1,177 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec.vector.reducesink; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.CompilationOpContext; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator.Counter; +import org.apache.hadoop.hive.ql.exec.TerminalOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow; +import org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow; +import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; +import org.apache.hadoop.hive.ql.exec.vector.VectorizationContextRegion; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression; +import org.apache.hadoop.hive.ql.exec.vector.keyseries.VectorKeySeriesSerialized; +import org.apache.hadoop.hive.ql.io.HiveKey; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.VectorReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.VectorReduceSinkInfo; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.ByteStream.Output; +import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe; +import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite; +import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinarySerializeWrite; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hive.common.util.HashCodeUtil; + +import com.google.common.base.Preconditions; + +/** + * This class is the UniformHash empty key operator class for native vectorized reduce sink. + * + * Since there is no key, we initialize the keyWritable once with an empty value. + */ +public class VectorReduceSinkEmptyKeyOperator extends VectorReduceSinkCommonOperator { + + private static final long serialVersionUID = 1L; + private static final String CLASS_NAME = VectorReduceSinkEmptyKeyOperator.class.getName(); + private static final Log LOG = LogFactory.getLog(CLASS_NAME); + + // The above members are initialized by the constructor and must not be + // transient. + //--------------------------------------------------------------------------- + + private transient boolean isKeyInitialized; + + /** Kryo ctor. */ + protected VectorReduceSinkEmptyKeyOperator() { + super(); + } + + public VectorReduceSinkEmptyKeyOperator(CompilationOpContext ctx) { + super(ctx); + } + + public VectorReduceSinkEmptyKeyOperator(CompilationOpContext ctx, + VectorizationContext vContext, OperatorDesc conf) throws HiveException { + super(ctx, vContext, conf); + + LOG.info("VectorReduceSinkEmptyKeyOperator constructor vectorReduceSinkInfo " + vectorReduceSinkInfo); + + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + + isKeyInitialized = false; + + } + + @Override + public void process(Object row, int tag) throws HiveException { + + try { + + VectorizedRowBatch batch = (VectorizedRowBatch) row; + + batchCounter++; + + if (batch.size == 0) { + if (LOG.isDebugEnabled()) { + LOG.debug(CLASS_NAME + " batch #" + batchCounter + " empty"); + } + return; + } + + if (!isKeyInitialized) { + isKeyInitialized = true; + Preconditions.checkState(isEmptyKey); + initializeEmptyKey(tag); + } + + // Perform any value expressions. Results will go into scratch columns. + if (reduceSinkValueExpressions != null) { + for (VectorExpression ve : reduceSinkValueExpressions) { + ve.evaluate(batch); + } + } + + final int size = batch.size; + if (!isEmptyValue) { + if (batch.selectedInUse) { + int[] selected = batch.selected; + for (int logical = 0; logical < size; logical++) { + final int batchIndex = selected[logical]; + + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, batchIndex); + + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + + collect(keyWritable, valueBytesWritable); + } + } else { + for (int batchIndex = 0; batchIndex < size; batchIndex++) { + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, batchIndex); + + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + + collect(keyWritable, valueBytesWritable); + } + } + } else { + + // Empty value, too. + for (int i = 0; i < size; i++) { + collect(keyWritable, valueBytesWritable); + } + } + } catch (Exception e) { + throw new HiveException(e); + } + } +} \ No newline at end of file diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java index 0bc1cd1..84fb9d3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java @@ -30,7 +30,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; /* - * Specialized class for native vectorized reduce sink that is reducing on a single long key column. + * Specialized class for native vectorized reduce sink that is reducing on a Uniform Hash + * single long key column. */ public class VectorReduceSinkLongOperator extends VectorReduceSinkUniformHashOperator { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java index 1cca94d..383cc90 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java @@ -29,8 +29,8 @@ import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite; /* - * Specialized class for native vectorized reduce sink that is reducing on multiple key columns - * (or a single non-long / non-string column). + * Specialized class for native vectorized reduce sink that is reducing on Uniform Hash + * multiple key columns (or a single non-long / non-string column). */ public class VectorReduceSinkMultiKeyOperator extends VectorReduceSinkUniformHashOperator { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java index 6312c44..597a401 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java @@ -69,7 +69,9 @@ import com.google.common.base.Preconditions; /** - * This class is uniform hash (common) operator class for native vectorized reduce sink. + * This class is the object hash (not Uniform Hash) operator class for native vectorized reduce sink. + * It takes the "object" hash code of bucket and/or partition keys (which are often subsets of the + * reduce key). If the bucket and partition keys are empty, the hash will be a random number. */ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOperator { @@ -91,6 +93,8 @@ // transient. //--------------------------------------------------------------------------- + private transient boolean isKeyInitialized; + protected transient Output keyOutput; protected transient VectorSerializeRow keyVectorSerializeRow; @@ -149,12 +153,17 @@ public VectorReduceSinkObjectHashOperator(CompilationOpContext ctx, protected void initializeOp(Configuration hconf) throws HiveException { super.initializeOp(hconf); - keyOutput = new Output(); - keyBinarySortableSerializeWrite.set(keyOutput); - keyVectorSerializeRow = - new VectorSerializeRow( - keyBinarySortableSerializeWrite); - keyVectorSerializeRow.init(reduceSinkKeyTypeInfos, reduceSinkKeyColumnMap); + if (!isEmptyKey) { + + // For this variation, we serialize the key without caring if it single Long, + // single String, multi-key, etc. + keyOutput = new Output(); + keyBinarySortableSerializeWrite.set(keyOutput); + keyVectorSerializeRow = + new VectorSerializeRow( + keyBinarySortableSerializeWrite); + keyVectorSerializeRow.init(reduceSinkKeyTypeInfos, reduceSinkKeyColumnMap); + } hasBuckets = false; isPartitioned = false; @@ -200,6 +209,13 @@ public void process(Object row, int tag) throws HiveException { return; } + if (!isKeyInitialized) { + isKeyInitialized = true; + if (isEmptyKey) { + initializeEmptyKey(tag); + } + } + // Perform any key expressions. Results will go into scratch columns. if (reduceSinkKeyExpressions != null) { for (VectorExpression ve : reduceSinkKeyExpressions) { @@ -260,25 +276,29 @@ public void process(Object row, int tag) throws HiveException { } } - keyBinarySortableSerializeWrite.reset(); - keyVectorSerializeRow.serializeWrite(batch, batchIndex); - - // One serialized key for 1 or more rows for the duplicate keys. - final int keyLength = keyOutput.getLength(); - if (tag == -1 || reduceSkipTag) { - keyWritable.set(keyOutput.getData(), 0, keyLength); - } else { - keyWritable.setSize(keyLength + 1); - System.arraycopy(keyOutput.getData(), 0, keyWritable.get(), 0, keyLength); - keyWritable.get()[keyLength] = reduceTagByte; + if (!isEmptyKey) { + keyBinarySortableSerializeWrite.reset(); + keyVectorSerializeRow.serializeWrite(batch, batchIndex); + + // One serialized key for 1 or more rows for the duplicate keys. + final int keyLength = keyOutput.getLength(); + if (tag == -1 || reduceSkipTag) { + keyWritable.set(keyOutput.getData(), 0, keyLength); + } else { + keyWritable.setSize(keyLength + 1); + System.arraycopy(keyOutput.getData(), 0, keyWritable.get(), 0, keyLength); + keyWritable.get()[keyLength] = reduceTagByte; + } + keyWritable.setDistKeyLength(keyLength); + keyWritable.setHashCode(hashCode); } - keyWritable.setDistKeyLength(keyLength); - keyWritable.setHashCode(hashCode); - - valueLazyBinarySerializeWrite.reset(); - valueVectorSerializeRow.serializeWrite(batch, batchIndex); - valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + if (!isEmptyValue) { + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, batchIndex); + + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + } collect(keyWritable, valueBytesWritable); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java index a838f4c..51e8531 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java @@ -30,7 +30,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; /* - * Specialized class for native vectorized reduce sink that is reducing on a single long key column. + * Specialized class for native vectorized reduce sink that is reducing on a Uniform Hash + * single long key column. */ public class VectorReduceSinkStringOperator extends VectorReduceSinkUniformHashOperator { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java index 2dfa721..5dcfde0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java @@ -66,8 +66,12 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hive.common.util.HashCodeUtil; +import com.google.common.base.Preconditions; + /** * This class is uniform hash (common) operator class for native vectorized reduce sink. + * There are variation operators for Long, String, and MultiKey. And, a special case operator + * for no key (VectorReduceSinkEmptyKeyOperator). */ public abstract class VectorReduceSinkUniformHashOperator extends VectorReduceSinkCommonOperator { @@ -105,6 +109,7 @@ public VectorReduceSinkUniformHashOperator(CompilationOpContext ctx, protected void initializeOp(Configuration hconf) throws HiveException { super.initializeOp(hconf); + Preconditions.checkState(!isEmptyKey); // Create all nulls key. try { Output nullKeyOutput = new Output(); @@ -155,10 +160,7 @@ public void process(Object row, int tag) throws HiveException { boolean selectedInUse = batch.selectedInUse; int[] selected = batch.selected; - int keyLength; int logical; - int end; - int batchIndex; do { if (serializedKeySeries.getCurrentIsAllNull()) { @@ -179,7 +181,7 @@ public void process(Object row, int tag) throws HiveException { // One serialized key for 1 or more rows for the duplicate keys. // LOG.info("reduceSkipTag " + reduceSkipTag + " tag " + tag + " reduceTagByte " + (int) reduceTagByte + " keyLength " + serializedKeySeries.getSerializedLength()); // LOG.info("process offset " + serializedKeySeries.getSerializedStart() + " length " + serializedKeySeries.getSerializedLength()); - keyLength = serializedKeySeries.getSerializedLength(); + final int keyLength = serializedKeySeries.getSerializedLength(); if (tag == -1 || reduceSkipTag) { keyWritable.set(serializedKeySeries.getSerializedBytes(), serializedKeySeries.getSerializedStart(), keyLength); @@ -194,17 +196,37 @@ public void process(Object row, int tag) throws HiveException { } logical = serializedKeySeries.getCurrentLogical(); - end = logical + serializedKeySeries.getCurrentDuplicateCount(); - do { - batchIndex = (selectedInUse ? selected[logical] : logical); + final int end = logical + serializedKeySeries.getCurrentDuplicateCount(); + if (!isEmptyValue) { + if (selectedInUse) { + do { + final int batchIndex = selected[logical]; + + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, batchIndex); + + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + + collect(keyWritable, valueBytesWritable); + } while (++logical < end); + } else { + do { + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, logical); - valueLazyBinarySerializeWrite.reset(); - valueVectorSerializeRow.serializeWrite(batch, batchIndex); + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); - valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + collect(keyWritable, valueBytesWritable); + } while (++logical < end); - collect(keyWritable, valueBytesWritable); - } while (++logical < end); + } + } else { + + // Empty value, too. + do { + collect(keyWritable, valueBytesWritable); + } while (++logical < end); + } if (!serializedKeySeries.next()) { break; diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index f0df2e9..f6ead6d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinOuterLongOperator; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinOuterMultiKeyOperator; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinOuterStringOperator; +import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkEmptyKeyOperator; import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkLongOperator; import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkMultiKeyOperator; import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkObjectHashOperator; @@ -2930,13 +2931,15 @@ private boolean canSpecializeMapJoin(Operator op, MapJoi Operator op, VectorizationContext vContext, ReduceSinkDesc desc, VectorReduceSinkInfo vectorReduceSinkInfo) throws HiveException { + VectorReduceSinkDesc vectorDesc = (VectorReduceSinkDesc) desc.getVectorDesc(); + Type[] reduceSinkKeyColumnVectorTypes = vectorReduceSinkInfo.getReduceSinkKeyColumnVectorTypes(); // By default, we can always use the multi-key class. VectorReduceSinkDesc.ReduceSinkKeyType reduceSinkKeyType = VectorReduceSinkDesc.ReduceSinkKeyType.MULTI_KEY; // Look for single column optimization. - if (reduceSinkKeyColumnVectorTypes.length == 1) { + if (reduceSinkKeyColumnVectorTypes != null && reduceSinkKeyColumnVectorTypes.length == 1) { LOG.info("Vectorizer vectorizeOperator groupby typeName " + vectorReduceSinkInfo.getReduceSinkKeyTypeInfos()[0]); Type columnVectorType = reduceSinkKeyColumnVectorTypes[0]; switch (columnVectorType) { @@ -2968,25 +2971,27 @@ private boolean canSpecializeMapJoin(Operator op, MapJoi Class> opClass = null; if (vectorReduceSinkInfo.getUseUniformHash()) { - switch (reduceSinkKeyType) { - case LONG: - opClass = VectorReduceSinkLongOperator.class; - break; - case STRING: - opClass = VectorReduceSinkStringOperator.class; - break; - case MULTI_KEY: - opClass = VectorReduceSinkMultiKeyOperator.class; - break; - default: - throw new HiveException("Unknown reduce sink key type " + reduceSinkKeyType); + if (vectorDesc.getIsEmptyKey()) { + opClass = VectorReduceSinkEmptyKeyOperator.class; + } else { + switch (reduceSinkKeyType) { + case LONG: + opClass = VectorReduceSinkLongOperator.class; + break; + case STRING: + opClass = VectorReduceSinkStringOperator.class; + break; + case MULTI_KEY: + opClass = VectorReduceSinkMultiKeyOperator.class; + break; + default: + throw new HiveException("Unknown reduce sink key type " + reduceSinkKeyType); + } } } else { opClass = VectorReduceSinkObjectHashOperator.class; } - VectorReduceSinkDesc vectorDesc = (VectorReduceSinkDesc) desc.getVectorDesc(); - vectorDesc.setReduceSinkKeyType(reduceSinkKeyType); vectorDesc.setVectorReduceSinkInfo(vectorReduceSinkInfo); @@ -3041,88 +3046,82 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, // So if we later decide not to specialize, we'll just waste any scratch columns allocated... List keysDescs = desc.getKeyCols(); - VectorExpression[] allKeyExpressions = vContext.getVectorExpressions(keysDescs); + final boolean isEmptyKey = (keysDescs.size() == 0); + if (!isEmptyKey) { - // Since a key expression can be a calculation and the key will go into a scratch column, - // we need the mapping and type information. - int[] reduceSinkKeyColumnMap = new int[allKeyExpressions.length]; - TypeInfo[] reduceSinkKeyTypeInfos = new TypeInfo[allKeyExpressions.length]; - Type[] reduceSinkKeyColumnVectorTypes = new Type[allKeyExpressions.length]; - ArrayList groupByKeyExpressionsList = new ArrayList(); - VectorExpression[] reduceSinkKeyExpressions; - for (int i = 0; i < reduceSinkKeyColumnMap.length; i++) { - VectorExpression ve = allKeyExpressions[i]; - reduceSinkKeyColumnMap[i] = ve.getOutputColumn(); - reduceSinkKeyTypeInfos[i] = keysDescs.get(i).getTypeInfo(); - reduceSinkKeyColumnVectorTypes[i] = - VectorizationContext.getColumnVectorTypeFromTypeInfo(reduceSinkKeyTypeInfos[i]); - if (!IdentityExpression.isColumnOnly(ve)) { - groupByKeyExpressionsList.add(ve); + VectorExpression[] allKeyExpressions = vContext.getVectorExpressions(keysDescs); + + final int[] reduceSinkKeyColumnMap = new int[allKeyExpressions.length]; + final TypeInfo[] reduceSinkKeyTypeInfos = new TypeInfo[allKeyExpressions.length]; + final Type[] reduceSinkKeyColumnVectorTypes = new Type[allKeyExpressions.length]; + final VectorExpression[] reduceSinkKeyExpressions; + + // Since a key expression can be a calculation and the key will go into a scratch column, + // we need the mapping and type information. + ArrayList groupByKeyExpressionsList = new ArrayList(); + for (int i = 0; i < reduceSinkKeyColumnMap.length; i++) { + VectorExpression ve = allKeyExpressions[i]; + reduceSinkKeyColumnMap[i] = ve.getOutputColumn(); + reduceSinkKeyTypeInfos[i] = keysDescs.get(i).getTypeInfo(); + reduceSinkKeyColumnVectorTypes[i] = + VectorizationContext.getColumnVectorTypeFromTypeInfo(reduceSinkKeyTypeInfos[i]); + if (!IdentityExpression.isColumnOnly(ve)) { + groupByKeyExpressionsList.add(ve); + } + } + if (groupByKeyExpressionsList.size() == 0) { + reduceSinkKeyExpressions = null; + } else { + reduceSinkKeyExpressions = groupByKeyExpressionsList.toArray(new VectorExpression[0]); } - } - if (groupByKeyExpressionsList.size() == 0) { - reduceSinkKeyExpressions = null; - } else { - reduceSinkKeyExpressions = groupByKeyExpressionsList.toArray(new VectorExpression[0]); + + vectorReduceSinkInfo.setReduceSinkKeyColumnMap(reduceSinkKeyColumnMap); + vectorReduceSinkInfo.setReduceSinkKeyTypeInfos(reduceSinkKeyTypeInfos); + vectorReduceSinkInfo.setReduceSinkKeyColumnVectorTypes(reduceSinkKeyColumnVectorTypes); + vectorReduceSinkInfo.setReduceSinkKeyExpressions(reduceSinkKeyExpressions); + } ArrayList valueDescs = desc.getValueCols(); - VectorExpression[] allValueExpressions = vContext.getVectorExpressions(valueDescs); - - int[] reduceSinkValueColumnMap = new int[valueDescs.size()]; - TypeInfo[] reduceSinkValueTypeInfos = new TypeInfo[valueDescs.size()]; - Type[] reduceSinkValueColumnVectorTypes = new Type[valueDescs.size()]; - ArrayList reduceSinkValueExpressionsList = new ArrayList(); - VectorExpression[] reduceSinkValueExpressions; - for (int i = 0; i < valueDescs.size(); ++i) { - VectorExpression ve = allValueExpressions[i]; - reduceSinkValueColumnMap[i] = ve.getOutputColumn(); - reduceSinkValueTypeInfos[i] = valueDescs.get(i).getTypeInfo(); - reduceSinkValueColumnVectorTypes[i] = - VectorizationContext.getColumnVectorTypeFromTypeInfo(reduceSinkValueTypeInfos[i]); - if (!IdentityExpression.isColumnOnly(ve)) { - reduceSinkValueExpressionsList.add(ve); + final boolean isEmptyValue = (valueDescs.size() == 0); + if (!isEmptyValue) { + VectorExpression[] allValueExpressions = vContext.getVectorExpressions(valueDescs); + + final int[] reduceSinkValueColumnMap = new int[allValueExpressions.length]; + final TypeInfo[] reduceSinkValueTypeInfos = new TypeInfo[allValueExpressions.length]; + final Type[] reduceSinkValueColumnVectorTypes = new Type[allValueExpressions.length]; + VectorExpression[] reduceSinkValueExpressions; + + ArrayList reduceSinkValueExpressionsList = new ArrayList(); + for (int i = 0; i < valueDescs.size(); ++i) { + VectorExpression ve = allValueExpressions[i]; + reduceSinkValueColumnMap[i] = ve.getOutputColumn(); + reduceSinkValueTypeInfos[i] = valueDescs.get(i).getTypeInfo(); + reduceSinkValueColumnVectorTypes[i] = + VectorizationContext.getColumnVectorTypeFromTypeInfo(reduceSinkValueTypeInfos[i]); + if (!IdentityExpression.isColumnOnly(ve)) { + reduceSinkValueExpressionsList.add(ve); + } + } + if (reduceSinkValueExpressionsList.size() == 0) { + reduceSinkValueExpressions = null; + } else { + reduceSinkValueExpressions = reduceSinkValueExpressionsList.toArray(new VectorExpression[0]); } - } - if (reduceSinkValueExpressionsList.size() == 0) { - reduceSinkValueExpressions = null; - } else { - reduceSinkValueExpressions = reduceSinkValueExpressionsList.toArray(new VectorExpression[0]); - } - vectorReduceSinkInfo.setReduceSinkKeyColumnMap(reduceSinkKeyColumnMap); - vectorReduceSinkInfo.setReduceSinkKeyTypeInfos(reduceSinkKeyTypeInfos); - vectorReduceSinkInfo.setReduceSinkKeyColumnVectorTypes(reduceSinkKeyColumnVectorTypes); - vectorReduceSinkInfo.setReduceSinkKeyExpressions(reduceSinkKeyExpressions); + vectorReduceSinkInfo.setReduceSinkValueColumnMap(reduceSinkValueColumnMap); + vectorReduceSinkInfo.setReduceSinkValueTypeInfos(reduceSinkValueTypeInfos); + vectorReduceSinkInfo.setReduceSinkValueColumnVectorTypes(reduceSinkValueColumnVectorTypes); + vectorReduceSinkInfo.setReduceSinkValueExpressions(reduceSinkValueExpressions); - vectorReduceSinkInfo.setReduceSinkValueColumnMap(reduceSinkValueColumnMap); - vectorReduceSinkInfo.setReduceSinkValueTypeInfos(reduceSinkValueTypeInfos); - vectorReduceSinkInfo.setReduceSinkValueColumnVectorTypes(reduceSinkValueColumnVectorTypes); - vectorReduceSinkInfo.setReduceSinkValueExpressions(reduceSinkValueExpressions); + } boolean useUniformHash = desc.getReducerTraits().contains(UNIFORM); vectorReduceSinkInfo.setUseUniformHash(useUniformHash); - boolean hasEmptyBuckets = false; - boolean hasNoPartitions = false; if (useUniformHash) { - // Check for unexpected conditions... - hasEmptyBuckets = - (desc.getBucketCols() != null && !desc.getBucketCols().isEmpty()) || - (desc.getPartitionCols().size() == 0); - - if (hasEmptyBuckets) { - LOG.info("Unexpected condition: UNIFORM hash and empty buckets"); - isUnexpectedCondition = true; - } - - hasNoPartitions = (desc.getPartitionCols() == null); - - if (hasNoPartitions) { - LOG.info("Unexpected condition: UNIFORM hash and no partitions"); - isUnexpectedCondition = true; - } + // NOTE: For Uniform Hash, when the key is empty, we will use the VectorReduceSinkEmptyKeyOperator instead. } else { @@ -3202,6 +3201,8 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, // Remember the condition variables for EXPLAIN regardless. vectorDesc.setIsVectorizationReduceSinkNativeEnabled(isVectorizationReduceSinkNativeEnabled); vectorDesc.setEngine(engine); + vectorDesc.setIsEmptyKey(isEmptyKey); + vectorDesc.setIsEmptyValue(isEmptyValue); vectorDesc.setHasTopN(hasTopN); vectorDesc.setHasDistinctColumns(hasDistinctColumns); vectorDesc.setIsKeyBinarySortable(isKeyBinarySortable); @@ -3214,7 +3215,6 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, // Many restrictions. if (!isVectorizationReduceSinkNativeEnabled || !isTezOrSpark || - (useUniformHash && (hasEmptyBuckets || hasNoPartitions)) || hasTopN || hasDistinctColumns || !isKeyBinarySortable || diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 654f3b1..f71aecf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -13195,7 +13195,11 @@ void buildPTFReduceSinkDetails(PartitionedTableFunctionDef tabDef, List partColList = tabDef.getPartition().getExpressions(); for (PTFExpressionDef colDef : partColList) { - ExprNodeDesc exprNode = colDef.getExprNode(); + final ExprNodeDesc exprNode = colDef.getExprNode(); + if (ExprNodeDescUtils.isConstant(exprNode)) { + // ignore partitioning by constants + continue; + } if (ExprNodeDescUtils.indexOf(exprNode, partCols) < 0) { partCols.add(exprNode); orderCols.add(exprNode); @@ -13213,10 +13217,15 @@ void buildPTFReduceSinkDetails(PartitionedTableFunctionDef tabDef, */ List orderColList = tabDef.getOrder().getExpressions(); for (int i = 0; i < orderColList.size(); i++) { - OrderExpressionDef colDef = orderColList.get(i); - char orderChar = colDef.getOrder() == PTFInvocationSpec.Order.ASC ? '+' : '-'; - char nullOrderChar = colDef.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST ? 'a' : 'z'; - int index = ExprNodeDescUtils.indexOf(colDef.getExprNode(), orderCols); + final OrderExpressionDef colDef = orderColList.get(i); + final char orderChar = colDef.getOrder() == PTFInvocationSpec.Order.ASC ? '+' : '-'; + final char nullOrderChar = colDef.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST ? 'a' : 'z'; + final ExprNodeDesc exprNode = colDef.getExprNode(); + if (ExprNodeDescUtils.isConstant(exprNode)) { + // ignore ordering by constants + continue; + } + int index = ExprNodeDescUtils.indexOf(exprNode, orderCols); if (index >= 0) { orderString.setCharAt(index, orderChar); nullOrderString.setCharAt(index, nullOrderChar); @@ -13353,7 +13362,10 @@ private Operator genReduceSinkPlanForWindowing(WindowingSpec spec, StringBuilder nullOrder = new StringBuilder(); for (PartitionExpression partCol : spec.getQueryPartitionSpec().getExpressions()) { - ExprNodeDesc partExpr = genExprNodeDesc(partCol.getExpression(), inputRR); + final ExprNodeDesc partExpr = genExprNodeDesc(partCol.getExpression(), inputRR); + if (ExprNodeDescUtils.isConstant(partExpr)) { + continue; + } if (ExprNodeDescUtils.indexOf(partExpr, partCols) < 0) { partCols.add(partExpr); orderCols.add(partExpr); @@ -13364,7 +13376,10 @@ private Operator genReduceSinkPlanForWindowing(WindowingSpec spec, if (spec.getQueryOrderSpec() != null) { for (OrderExpression orderCol : spec.getQueryOrderSpec().getExpressions()) { - ExprNodeDesc orderExpr = genExprNodeDesc(orderCol.getExpression(), inputRR); + final ExprNodeDesc orderExpr = genExprNodeDesc(orderCol.getExpression(), inputRR); + if (ExprNodeDescUtils.isConstant(orderExpr)) { + continue; + } char orderChar = orderCol.getOrder() == PTFInvocationSpec.Order.ASC ? '+' : '-'; char nullOrderChar = orderCol.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST ? 'a' : 'z'; int index = ExprNodeDescUtils.indexOf(orderExpr, orderCols); 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 79d19b5..c0c048b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java @@ -527,6 +527,58 @@ public ReduceSinkOperatorExplainVectorization(ReduceSinkDesc reduceSinkDesc, Vec return vectorExpressionsToStringList(vectorReduceSinkInfo.getReduceSinkValueExpressions()); } + @Explain(vectorization = Vectorization.DETAIL, displayName = "keyColumns", explainLevels = { Level.DEFAULT, Level.EXTENDED }) + public String getKeyColumns() { + if (!isNative) { + return null; + } + int[] keyColumnMap = vectorReduceSinkInfo.getReduceSinkKeyColumnMap(); + if (keyColumnMap == null) { + // Always show an array. + keyColumnMap = new int[0]; + } + return Arrays.toString(keyColumnMap); + } + + @Explain(vectorization = Vectorization.DETAIL, displayName = "valueColumns", explainLevels = { Level.DEFAULT, Level.EXTENDED }) + public String getValueColumns() { + if (!isNative) { + return null; + } + int[] valueColumnMap = vectorReduceSinkInfo.getReduceSinkValueColumnMap(); + if (valueColumnMap == null) { + // Always show an array. + valueColumnMap = new int[0]; + } + return Arrays.toString(valueColumnMap); + } + + @Explain(vectorization = Vectorization.DETAIL, displayName = "bucketColumns", explainLevels = { Level.DEFAULT, Level.EXTENDED }) + public String getBucketColumns() { + if (!isNative) { + return null; + } + int[] bucketColumnMap = vectorReduceSinkInfo.getReduceSinkBucketColumnMap(); + if (bucketColumnMap == null || bucketColumnMap.length == 0) { + // Suppress empty column map. + return null; + } + return Arrays.toString(bucketColumnMap); + } + + @Explain(vectorization = Vectorization.DETAIL, displayName = "partitionColumns", explainLevels = { Level.DEFAULT, Level.EXTENDED }) + public String getPartitionColumns() { + if (!isNative) { + return null; + } + int[] partitionColumnMap = vectorReduceSinkInfo.getReduceSinkPartitionColumnMap(); + if (partitionColumnMap == null || partitionColumnMap.length == 0) { + // Suppress empty column map. + return null; + } + return Arrays.toString(partitionColumnMap); + } + private VectorizationCondition[] createNativeConditions() { boolean enabled = vectorReduceSinkDesc.getIsVectorizationReduceSinkNativeEnabled(); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java index d6230af..6f2bee5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java @@ -64,6 +64,8 @@ public VectorReduceSinkInfo getVectorReduceSinkInfo() { private boolean isVectorizationReduceSinkNativeEnabled; private String engine; + private boolean isEmptyKey; + private boolean isEmptyValue; private boolean hasTopN; private boolean hasDistinctColumns; private boolean isKeyBinarySortable; @@ -85,6 +87,18 @@ public void setEngine(String engine) { public String getEngine() { return engine; } + public void setIsEmptyKey(boolean isEmptyKey) { + this.isEmptyKey = isEmptyKey; + } + public boolean getIsEmptyKey() { + return isEmptyKey; + } + public void setIsEmptyValue(boolean isEmptyValue) { + this.isEmptyValue = isEmptyValue; + } + public boolean getIsEmptyValue() { + return isEmptyValue; + } public void setHasTopN(boolean hasTopN) { this.hasTopN = hasTopN; } diff --git ql/src/test/queries/clientpositive/vector_windowing_navfn.q ql/src/test/queries/clientpositive/vector_windowing_navfn.q new file mode 100644 index 0000000..9acbe97 --- /dev/null +++ ql/src/test/queries/clientpositive/vector_windowing_navfn.q @@ -0,0 +1,134 @@ +set hive.explain.user=false; +set hive.cli.print.header=true; +SET hive.vectorized.execution.enabled=true; +set hive.fetch.task.conversion=none; + +drop table over10k; + +create table over10k( + 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/over10k' into table over10k; + +explain vectorization detail +select row_number() over() from src where key = '238'; + +select row_number() over() from src where key = '238'; + +explain vectorization detail +select s, row_number() over (partition by d order by `dec`) from over10k limit 100; + +select s, row_number() over (partition by d order by `dec`) from over10k limit 100; + +explain vectorization detail +select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100; + +select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100; + +explain vectorization detail +select i, lag(`dec`) over (partition by i order by s,i,`dec`) from over10k limit 100; + +select i, lag(`dec`) over (partition by i order by s,i,`dec`) from over10k limit 100; + +explain vectorization detail +select s, last_value(t) over (partition by d order by f) from over10k limit 100; + +select s, last_value(t) over (partition by d order by f) from over10k limit 100; + +explain vectorization detail +select s, first_value(s) over (partition by bo order by s) from over10k limit 100; + +select s, first_value(s) over (partition by bo order by s) from over10k limit 100; + +explain vectorization detail +select t, s, i, last_value(i) over (partition by t order by s) +from over10k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; + +-- select t, s, i, last_value(i) over (partition by t order by s) +-- from over10k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; + +drop table if exists wtest; +create table wtest as +select a, b +from +( +SELECT explode( + map( + 3, array(1,2,3,4,5), + 1, array(int(null),int(null),int(null), int(null), int(null)), + 2, array(1,null,2, null, 3) + ) + ) as (a,barr) FROM (select * from src limit 1) s + ) s1 lateral view explode(barr) arr as b; + +explain vectorization detail +select a, b, +first_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest; + +select a, b, +first_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest; + +explain vectorization detail +select a, b, +first_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest; + +select a, b, +first_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest; + +explain vectorization detail +select a, b, +last_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest; + +select a, b, +last_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest; + +explain vectorization detail +select a, b, +last_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest; + +select a, b, +last_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest; diff --git ql/src/test/queries/clientpositive/vectorization_limit.q ql/src/test/queries/clientpositive/vectorization_limit.q index a4c54f2..8044484 100644 --- ql/src/test/queries/clientpositive/vectorization_limit.q +++ ql/src/test/queries/clientpositive/vectorization_limit.q @@ -3,6 +3,8 @@ set hive.explain.user=false; SET hive.vectorized.execution.enabled=true; set hive.fetch.task.conversion=none; +-- SORT_QUERY_RESULTS + explain vectorization SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; @@ -11,31 +13,31 @@ set hive.limit.pushdown.memory.usage=0.3f; -- HIVE-3562 Some limit can be pushed down to map stage - c/p parts from limit_pushdown -explain vectorization expression +explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; -- deduped RS -explain vectorization expression +explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; -- distincts -explain vectorization expression +explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20; select distinct(ctinyint) from alltypesorc limit 20; -explain vectorization expression +explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20; select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20; -- limit zero -explain vectorization expression +explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; -- 2MR (applied to last RS) -explain vectorization expression +explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; diff --git ql/src/test/queries/clientpositive/windowing_navfn.q ql/src/test/queries/clientpositive/windowing_navfn.q index f2ec9fc..7c27e22 100644 --- ql/src/test/queries/clientpositive/windowing_navfn.q +++ ql/src/test/queries/clientpositive/windowing_navfn.q @@ -17,6 +17,8 @@ create table over10k( load data local inpath '../../data/files/over10k' into table over10k; +explain select row_number() over() from src where key = '238'; + select row_number() over() from src where key = '238'; select s, row_number() over (partition by d order by `dec`) from over10k limit 100; diff --git ql/src/test/results/clientpositive/llap/vector_groupby_grouping_id3.q.out ql/src/test/results/clientpositive/llap/vector_groupby_grouping_id3.q.out index cfc87fb..3892a28 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby_grouping_id3.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby_grouping_id3.q.out @@ -100,9 +100,11 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int), _col1 (type: int), 1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [0, 1, 4] keyExpressions: ConstantVectorExpression(val 1) -> 4:long native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [3] Statistics: Num rows: 6 Data size: 40 Basic stats: COMPLETE Column stats: NONE value expressions: _col3 (type: bigint) Execution mode: vectorized, llap @@ -265,8 +267,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [0, 1, 2] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [3] Statistics: Num rows: 12 Data size: 80 Basic stats: COMPLETE Column stats: NONE value expressions: _col3 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_inner_join.q.out ql/src/test/results/clientpositive/llap/vector_inner_join.q.out index fc7b9a3..da6fdaa 100644 --- ql/src/test/results/clientpositive/llap/vector_inner_join.q.out +++ ql/src/test/results/clientpositive/llap/vector_inner_join.q.out @@ -152,8 +152,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -316,8 +318,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -510,8 +514,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -598,8 +604,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -831,8 +839,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -995,8 +1005,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1159,8 +1171,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1247,8 +1261,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1411,8 +1427,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out index 95dcba9..8d40a6d 100644 --- ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out +++ ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out @@ -111,24 +111,24 @@ Stage-0 Reducer 2 llap File Output Operator [FS_10] Merge Join Operator [MERGEJOIN_21] (rows=2 width=431) - Conds:RS_23._col2=RS_28._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + Conds:RS_6._col2=RS_7._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] <-Map 1 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_23] + SHUFFLE [RS_6] PartitionCols:_col2 - Select Operator [SEL_22] (rows=2 width=134) + Select Operator [SEL_2] (rows=2 width=134) Output:["_col0","_col1","_col2"] TableScan [TS_0] (rows=2 width=236) default@char_tbl1,c1,Tbl:COMPLETE,Col:PARTIAL,Output:["name","age"] - Dynamic Partitioning Event Operator [EVENT_26] (rows=1 width=134) - Group By Operator [GBY_25] (rows=1 width=134) + Dynamic Partitioning Event Operator [EVENT_20] (rows=1 width=134) + Group By Operator [GBY_19] (rows=1 width=134) Output:["_col0"],keys:_col0 - Select Operator [SEL_24] (rows=2 width=134) + Select Operator [SEL_18] (rows=2 width=134) Output:["_col0"] - Please refer to the previous Select Operator [SEL_22] + Please refer to the previous Select Operator [SEL_2] <-Map 3 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_28] + SHUFFLE [RS_7] PartitionCols:_col2 - Select Operator [SEL_27] (rows=2 width=89) + Select Operator [SEL_5] (rows=2 width=89) Output:["_col0","_col1","_col2"] TableScan [TS_3] (rows=2 width=190) default@char_tbl2,c2,Tbl:COMPLETE,Col:PARTIAL,Output:["name","age"] diff --git ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out index 6c6d0f3..145dea7 100644 --- ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out @@ -5910,8 +5910,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -5968,8 +5970,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6101,8 +6105,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6159,8 +6165,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6294,8 +6302,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6352,8 +6362,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6487,8 +6499,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6545,8 +6559,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6683,8 +6699,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6741,8 +6759,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6876,8 +6896,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6937,8 +6959,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7072,8 +7096,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7133,8 +7159,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7265,8 +7293,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7318,8 +7348,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7450,8 +7482,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7508,8 +7542,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7654,8 +7690,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7712,9 +7750,11 @@ STAGE PLANS: Map-reduce partition columns: (2 * _col0) (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] keyExpressions: LongScalarMultiplyLongColumn(val 2, col 0) -> 1:long native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7856,8 +7896,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [2, 3] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -7897,8 +7939,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -7956,8 +8000,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8099,8 +8145,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8157,8 +8205,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int), _col1 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8301,8 +8351,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8359,8 +8411,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8417,8 +8471,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8556,8 +8612,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8588,8 +8646,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8639,8 +8699,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8773,8 +8835,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8805,8 +8869,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8856,8 +8922,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9010,8 +9078,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9061,8 +9131,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9093,8 +9165,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9247,8 +9321,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9298,8 +9374,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9330,8 +9408,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9486,8 +9566,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9537,8 +9619,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9569,8 +9653,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9772,8 +9858,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 2475 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9830,8 +9918,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9862,8 +9952,10 @@ STAGE PLANS: Map-reduce partition columns: value (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10093,8 +10185,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10189,8 +10283,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10247,8 +10343,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10383,8 +10481,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10441,8 +10541,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10579,8 +10681,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10637,8 +10741,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10775,8 +10881,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10833,8 +10941,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10974,8 +11084,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11032,8 +11144,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11167,8 +11281,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11231,8 +11347,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11366,8 +11484,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11430,8 +11550,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11562,8 +11684,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11618,8 +11742,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11753,8 +11879,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11811,8 +11939,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11960,8 +12090,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12018,9 +12150,11 @@ STAGE PLANS: Map-reduce partition columns: (2 * _col0) (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] keyExpressions: LongScalarMultiplyLongColumn(val 2, col 0) -> 1:long native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12162,8 +12296,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [2, 3] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -12203,8 +12339,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -12262,8 +12400,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12408,8 +12548,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12466,8 +12608,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int), _col1 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12610,8 +12754,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12668,8 +12814,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12726,8 +12874,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12865,8 +13015,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12897,8 +13049,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12948,8 +13102,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13082,8 +13238,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13114,8 +13272,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13165,8 +13325,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13319,8 +13481,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13370,8 +13534,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13402,8 +13568,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13556,8 +13724,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13607,8 +13777,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13639,8 +13811,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13795,8 +13969,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13846,8 +14022,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13878,8 +14056,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14087,8 +14267,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 2475 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14145,8 +14327,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14177,8 +14361,10 @@ STAGE PLANS: Map-reduce partition columns: value (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14411,8 +14597,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14507,8 +14695,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14565,8 +14755,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14701,8 +14893,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14759,8 +14953,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14897,8 +15093,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14955,8 +15153,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15093,8 +15293,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15151,8 +15353,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15292,8 +15496,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15350,8 +15556,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15485,8 +15693,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15549,8 +15759,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15684,8 +15896,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15748,8 +15962,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15880,8 +16096,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15936,8 +16154,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16071,8 +16291,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16129,8 +16351,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16278,8 +16502,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16336,9 +16562,11 @@ STAGE PLANS: Map-reduce partition columns: (2 * _col0) (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] keyExpressions: LongScalarMultiplyLongColumn(val 2, col 0) -> 1:long native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16480,8 +16708,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [2, 3] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -16521,8 +16751,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -16580,8 +16812,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16726,8 +16960,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16784,8 +17020,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int), _col1 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16928,8 +17166,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16986,8 +17226,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17044,8 +17286,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17183,8 +17427,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17215,8 +17461,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17266,8 +17514,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17400,8 +17650,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17432,8 +17684,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17483,8 +17737,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17637,8 +17893,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17688,8 +17946,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17720,8 +17980,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17874,8 +18136,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17925,8 +18189,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17957,8 +18223,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18113,8 +18381,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18164,8 +18434,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18196,8 +18468,10 @@ STAGE PLANS: Map-reduce partition columns: key (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18405,8 +18679,10 @@ STAGE PLANS: sort order: + Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 2475 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18463,8 +18739,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18495,8 +18773,10 @@ STAGE PLANS: Map-reduce partition columns: value (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18729,8 +19009,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out index 6edc474..10051d1 100644 --- ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out +++ ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out @@ -21,7 +21,7 @@ STAGE PLANS: alias: alltypesorc Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (cint) IN (ctinyint, cbigint) (type: boolean) + predicate: (cint) IN (UDFToInteger(ctinyint), UDFToInteger(cbigint)) (type: boolean) Statistics: Num rows: 6144 Data size: 1546640 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean) @@ -40,7 +40,7 @@ STAGE PLANS: enabled: true enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - notVectorizedReason: Predicate expression for FILTER operator: Cannot vectorize IN() - casting a column is not supported. Column type is int but the common type is bigint + notVectorizedReason: Predicate expression for FILTER operator: Vectorizing IN expression only supported for constant values vectorized: false Stage: Stage-0 diff --git ql/src/test/results/clientpositive/llap/vector_order_null.q.out ql/src/test/results/clientpositive/llap/vector_order_null.q.out index 9df6f7a..7ddf165 100644 --- ql/src/test/results/clientpositive/llap/vector_order_null.q.out +++ ql/src/test/results/clientpositive/llap/vector_order_null.q.out @@ -104,8 +104,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -222,8 +224,10 @@ STAGE PLANS: sort order: -+ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -340,8 +344,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1, 0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -458,8 +464,10 @@ STAGE PLANS: sort order: -+ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1, 0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -576,8 +584,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -694,8 +704,10 @@ STAGE PLANS: sort order: -+ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -812,8 +824,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1, 0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -930,8 +944,10 @@ STAGE PLANS: sort order: -+ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1, 0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -1048,8 +1064,10 @@ STAGE PLANS: sort order: +- Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -1166,8 +1184,10 @@ STAGE PLANS: sort order: -- Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1, 0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -1284,8 +1304,10 @@ STAGE PLANS: sort order: ++ Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [1, 0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs diff --git ql/src/test/results/clientpositive/llap/vector_outer_join0.q.out ql/src/test/results/clientpositive/llap/vector_outer_join0.q.out index 5c3f0e0..545bc0a 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join0.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join0.q.out @@ -164,8 +164,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1] Statistics: Num rows: 6 Data size: 550 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized, llap @@ -250,8 +252,10 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0] Statistics: Num rows: 6 Data size: 544 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out index c089d00..9afd1c0 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out @@ -328,8 +328,10 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [2] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0, 1, 3, 4, 5, 6, 7, 8, 9, 10, 11] Statistics: Num rows: 15 Data size: 3697 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean) Execution mode: vectorized, llap @@ -496,8 +498,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: tinyint) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 15 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -742,8 +746,10 @@ STAGE PLANS: sort order: Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0, 1] Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap @@ -783,8 +789,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [2] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 15 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -823,8 +831,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: tinyint) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 15 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out index f1319a9..c90f709 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out @@ -328,8 +328,10 @@ STAGE PLANS: sort order: Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [0, 1] Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap @@ -369,8 +371,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [2] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 20 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -409,8 +413,10 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: bigint) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [3] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 20 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out index 9929550..0e47836 100644 --- ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out +++ ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out @@ -142,8 +142,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: double) Execution mode: vectorized, llap @@ -367,8 +369,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -592,8 +597,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -811,8 +819,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: double) Execution mode: vectorized, llap @@ -1004,8 +1014,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -1197,8 +1210,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -1390,8 +1406,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -1625,8 +1644,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 2] Statistics: Num rows: 40 Data size: 12792 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: decimal(38,18)) Execution mode: vectorized, llap @@ -1818,8 +1839,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 12792 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: decimal(38,18)) Execution mode: vectorized, llap @@ -2031,8 +2055,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_bigint (type: bigint) Execution mode: vectorized, llap @@ -2224,8 +2250,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_bigint (type: bigint) Execution mode: vectorized, llap @@ -2411,8 +2440,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -2575,8 +2606,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -2739,9 +2773,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), CASE WHEN ((p_mfgr = 'Manufacturer#2')) THEN (2000-01-01 00:00:00.0) ELSE (null) END (type: timestamp) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [0, 4] keyExpressions: VectorUDFAdaptor(CASE WHEN ((p_mfgr = 'Manufacturer#2')) THEN (2000-01-01 00:00:00.0) ELSE (null) END)(children: StringGroupColEqualStringScalar(col 0, val Manufacturer#2) -> 3:boolean) -> 4:timestamp native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2905,9 +2941,12 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), CASE WHEN ((p_mfgr = 'Manufacturer#2')) THEN (2000-01-01 00:00:00.0) ELSE (null) END (type: timestamp) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 4, 1] keyExpressions: VectorUDFAdaptor(CASE WHEN ((p_mfgr = 'Manufacturer#2')) THEN (2000-01-01 00:00:00.0) ELSE (null) END)(children: StringGroupColEqualStringScalar(col 0, val Manufacturer#2) -> 3:boolean) -> 4:timestamp native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0, 5] + valueColumns: [2] Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out index c036d69..6b4ca6c 100644 --- ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out +++ ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out @@ -62,28 +62,44 @@ STAGE PLANS: TableScan alias: orc_decimal Statistics: Num rows: 4 Data size: 448 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0] Filter Operator - predicate: (id) IN ('100000000', '200000000') (type: boolean) + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: FilterDoubleColumnInList(col 1, values [1.0E8, 2.0E8])(children: CastDecimalToDouble(col 0) -> 1:double) -> boolean + predicate: (UDFToDouble(id)) IN (1.0E8, 2.0E8) (type: boolean) Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: id (type: decimal(18,0)) outputColumnNames: _col0 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumns: [0] Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: false Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: enabled: true enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true + groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - notVectorizedReason: Predicate expression for FILTER operator: Cannot vectorize IN() - casting a column is not supported. Column type is decimal(18,0) but the common type is string - vectorized: false + allNative: false + usesVectorUDFAdaptor: false + vectorized: true Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/llap/vector_windowing_navfn.q.out ql/src/test/results/clientpositive/llap/vector_windowing_navfn.q.out new file mode 100644 index 0000000..580f5f5 --- /dev/null +++ ql/src/test/results/clientpositive/llap/vector_windowing_navfn.q.out @@ -0,0 +1,2105 @@ +PREHOOK: query: drop table over10k +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table over10k +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table over10k( + 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 +PREHOOK: Output: database:default +PREHOOK: Output: default@over10k +POSTHOOK: query: create table over10k( + 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: database:default +POSTHOOK: Output: default@over10k +PREHOOK: query: load data local inpath '../../data/files/over10k' into table over10k +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@over10k +POSTHOOK: query: load data local inpath '../../data/files/over10k' into table over10k +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@over10k +PREHOOK: query: explain vectorization detail +select row_number() over() from src where key = '238' +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select row_number() over() from src where key = '238' +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1] + Filter Operator + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: FilterStringGroupColEqualStringScalar(col 0, val 238) -> boolean + predicate: (key = '238') (type: boolean) + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Reduce Sink Vectorization: + className: VectorReduceSinkEmptyKeyOperator + keyColumns: [] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0] + dataColumns: key:string, value:string + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: 0 ASC NULLS FIRST + partition by: 0 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 2 Data size: 174 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: row_number_window_0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select row_number() over() from src where key = '238' +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select row_number() over() from src where key = '238' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +row_number_window_0 +1 +2 +PREHOOK: query: explain vectorization detail +select s, row_number() over (partition by d order by `dec`) from over10k limit 100 +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select s, row_number() over (partition by d order by `dec`) from over10k limit 100 +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: over10k + Statistics: Num rows: 4625 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + Reduce Output Operator + key expressions: d (type: double), dec (type: decimal(4,2)) + sort order: ++ + Map-reduce partition columns: d (type: double) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [5, 9] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [5] + valueColumns: [7] + Statistics: Num rows: 4625 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + value expressions: s (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 11 + includeColumns: [5, 7, 9] + dataColumns: t:tinyint, si:smallint, i:int, b:bigint, f:float, d:double, bo:boolean, s:string, ts:timestamp, dec:decimal(4,2), bin:binary + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: double), VALUE._col6 (type: string), KEY.reducesinkkey1 (type: decimal(4,2)) + outputColumnNames: _col5, _col7, _col9 + Statistics: Num rows: 4625 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col5: double, _col7: string, _col9: decimal(4,2) + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col9 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4625 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col7 (type: string), row_number_window_0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4625 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 22000 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 22000 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 100 + Processor Tree: + ListSink + +PREHOOK: query: select s, row_number() over (partition by d order by `dec`) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select s, row_number() over (partition by d order by `dec`) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +s row_number_window_0 +calvin miller 1 +yuri laertes 1 +wendy quirinius 2 +holly polk 3 +nick steinbeck 1 +tom young 1 +priscilla quirinius 2 +katie brown 3 +xavier robinson 1 +gabriella quirinius 2 +katie falkner 3 +ethan carson 1 +victor johnson 2 +zach white 1 +jessica white 2 +jessica king 3 +victor davidson 1 +quinn falkner 2 +holly falkner 3 +holly young 1 +xavier steinbeck 1 +nick robinson 2 +irene king 1 +quinn zipper 1 +priscilla miller 1 +yuri miller 2 +wendy zipper 3 +zach steinbeck 1 +fred nixon 1 +katie brown 1 +nick davidson 1 +gabriella davidson 1 +zach carson 2 +wendy king 1 +tom xylophone 2 +holly hernandez 3 +jessica quirinius 4 +gabriella brown 1 +quinn johnson 2 +yuri zipper 3 +david robinson 1 +mike nixon 2 +rachel davidson 1 +gabriella white 2 +yuri garcia 1 +yuri zipper 2 +katie hernandez 1 +alice king 2 +jessica steinbeck 3 +quinn davidson 1 +katie ovid 2 +priscilla young 3 +quinn van buren 4 +victor steinbeck 5 +gabriella brown 1 +zach laertes 1 +jessica ichabod 2 +ethan miller 1 +irene carson 2 +priscilla zipper 3 +irene falkner 4 +tom robinson 5 +katie polk 1 +xavier laertes 2 +sarah davidson 3 +nick white 4 +nick polk 1 +alice ichabod 2 +luke brown 1 +wendy allen 2 +gabriella robinson 3 +holly steinbeck 1 +calvin ichabod 2 +holly van buren 1 +tom nixon 2 +gabriella carson 3 +mike brown 1 +katie laertes 2 +zach garcia 1 +oscar nixon 2 +tom polk 1 +mike allen 1 +alice johnson 1 +yuri young 1 +holly robinson 2 +priscilla thompson 3 +rachel carson 1 +gabriella laertes 1 +victor brown 2 +holly allen 1 +bob carson 2 +rachel carson 1 +fred nixon 2 +priscilla brown 1 +alice nixon 2 +victor falkner 1 +david garcia 1 +holly hernandez 2 +tom white 3 +rachel ellison 1 +PREHOOK: query: explain vectorization detail +select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100 +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100 +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: over10k + Statistics: Num rows: 4799 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + Reduce Output Operator + key expressions: bin (type: binary), d (type: double), i (type: int) + sort order: ++- + Map-reduce partition columns: bin (type: binary) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [10, 5, 2] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [10] + valueColumns: [7] + Statistics: Num rows: 4799 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + value expressions: s (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 11 + includeColumns: [2, 5, 7, 10] + dataColumns: t:tinyint, si:smallint, i:int, b:bigint, f:float, d:double, bo:boolean, s:string, ts:timestamp, dec:decimal(4,2), bin:binary + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey2 (type: int), KEY.reducesinkkey1 (type: double), VALUE._col5 (type: string), KEY.reducesinkkey0 (type: binary) + outputColumnNames: _col2, _col5, _col7, _col10 + Statistics: Num rows: 4799 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: int, _col5: double, _col7: string, _col10: binary + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST, _col2 DESC NULLS LAST + partition by: _col10 + raw input shape: + window functions: + window function definition + alias: lead_window_0 + arguments: _col7 + name: lead + window function: GenericUDAFLeadEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4799 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: int), lead_window_0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4799 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 21200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 21200 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 100 + Processor Tree: + ListSink + +PREHOOK: query: select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +i lead_window_0 +65773 gabriella van buren +65727 quinn steinbeck +65649 katie laertes +65643 luke young +65712 xavier white +65554 oscar garcia +65715 katie ichabod +65737 mike garcia +65641 katie thompson +65674 zach falkner +65627 katie ovid +65628 irene davidson +65657 wendy van buren +65686 yuri ichabod +65594 ethan underhill +65615 zach young +65708 calvin falkner +65674 nick xylophone +65717 tom hernandez +65713 tom ichabod +65681 priscilla ichabod +65654 victor laertes +65580 victor van buren +65711 holly xylophone +65727 david garcia +65692 yuri ovid +65760 oscar xylophone +65545 luke falkner +65653 alice falkner +65773 jessica robinson +65756 fred davidson +65698 tom hernandez +65659 irene ellison +65662 holly robinson +65610 mike garcia +65537 yuri hernandez +65601 ulysses garcia +65545 mike brown +65538 ethan nixon +65551 yuri falkner +65709 rachel robinson +65774 yuri polk +65713 yuri nixon +65613 sarah thompson +65693 nick nixon +65650 ethan carson +65781 oscar king +65675 priscilla ichabod +65541 jessica thompson +65719 mike polk +65694 luke brown +65538 victor young +65746 oscar nixon +65564 tom zipper +65708 irene ellison +65696 alice davidson +65596 jessica garcia +65586 victor miller +65696 holly white +65600 david robinson +65785 jessica davidson +65606 ulysses brown +65734 ethan underhill +65546 ethan miller +65578 tom thompson +65776 holly steinbeck +65741 yuri underhill +65770 priscilla king +65562 bob white +65605 victor van buren +65764 bob allen +65725 david underhill +65700 holly king +65648 victor ovid +65733 priscilla xylophone +65682 katie miller +65694 victor xylophone +65553 mike steinbeck +65635 holly laertes +65537 katie steinbeck +65761 zach white +65747 ethan falkner +65601 ulysses king +65590 rachel davidson +65732 irene young +65642 bob thompson +65570 tom xylophone +65692 sarah davidson +65760 tom laertes +65784 yuri johnson +65630 zach laertes +65774 nick polk +65788 quinn xylophone +65595 yuri white +65765 xavier ellison +65541 jessica ichabod +65711 tom steinbeck +65536 alice hernandez +65545 tom zipper +65789 ulysses hernandez +PREHOOK: query: explain vectorization detail +select i, lag(`dec`) over (partition by i order by s,i,`dec`) from over10k limit 100 +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select i, lag(`dec`) over (partition by i order by s,i,`dec`) from over10k limit 100 +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: over10k + Statistics: Num rows: 4710 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + Reduce Output Operator + key expressions: i (type: int), s (type: string), dec (type: decimal(4,2)) + sort order: +++ + Map-reduce partition columns: i (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 7, 9] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [] + Statistics: Num rows: 4710 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 11 + includeColumns: [2, 7, 9] + dataColumns: t:tinyint, si:smallint, i:int, b:bigint, f:float, d:double, bo:boolean, s:string, ts:timestamp, dec:decimal(4,2), bin:binary + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: decimal(4,2)) + outputColumnNames: _col2, _col7, _col9 + Statistics: Num rows: 4710 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: int, _col7: string, _col9: decimal(4,2) + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col7 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col9 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: lag_window_0 + arguments: _col9 + name: lag + window function: GenericUDAFLagEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4710 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: int), lag_window_0 (type: decimal(4,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4710 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 21600 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 21600 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 100 + Processor Tree: + ListSink + +PREHOOK: query: select i, lag(`dec`) over (partition by i order by s,i,`dec`) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select i, lag(`dec`) over (partition by i order by s,i,`dec`) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +i lag_window_0 +65536 NULL +65536 31.09 +65536 1.33 +65536 86.04 +65536 52.78 +65536 93.68 +65536 98.42 +65536 0.93 +65536 83.48 +65536 75.70 +65536 88.04 +65536 94.09 +65536 33.45 +65536 44.41 +65536 22.15 +65536 20.50 +65536 58.86 +65536 30.91 +65536 74.47 +65536 11.31 +65536 59.67 +65536 63.08 +65536 5.53 +65536 95.99 +65536 59.58 +65536 32.68 +65536 16.85 +65536 4.34 +65536 28.49 +65536 80.26 +65536 35.07 +65536 95.88 +65536 30.60 +65536 46.97 +65536 58.80 +65536 5.72 +65536 29.27 +65536 62.25 +65536 45.25 +65536 85.25 +65536 30.25 +65536 65.17 +65536 1.05 +65536 39.32 +65536 32.03 +65537 NULL +65537 14.48 +65537 95.64 +65537 4.49 +65537 11.87 +65537 89.52 +65537 56.83 +65537 99.34 +65537 7.72 +65537 94.52 +65537 35.86 +65537 47.75 +65537 1.12 +65537 52.90 +65537 53.92 +65537 43.45 +65537 7.52 +65537 91.35 +65537 56.13 +65537 51.91 +65537 81.04 +65537 19.44 +65537 8.63 +65537 29.01 +65537 56.48 +65537 83.21 +65537 56.52 +65537 36.60 +65537 59.70 +65537 80.14 +65537 66.30 +65537 94.87 +65537 40.92 +65537 25.20 +65537 7.36 +65538 NULL +65538 53.35 +65538 54.64 +65538 76.67 +65538 15.17 +65538 1.20 +65538 13.71 +65538 81.59 +65538 43.33 +65538 30.27 +65538 8.91 +65538 95.81 +65538 92.44 +65538 98.11 +65538 18.02 +65538 78.41 +65538 35.14 +65538 9.53 +65538 48.61 +65538 2.03 +PREHOOK: query: explain vectorization detail +select s, last_value(t) over (partition by d order by f) from over10k limit 100 +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select s, last_value(t) over (partition by d order by f) from over10k limit 100 +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: over10k + Statistics: Num rows: 8771 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + Reduce Output Operator + key expressions: d (type: double), f (type: float) + sort order: ++ + Map-reduce partition columns: d (type: double) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [5, 4] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [5] + valueColumns: [0, 7] + Statistics: Num rows: 8771 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + value expressions: t (type: tinyint), s (type: string) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 11 + includeColumns: [0, 4, 5, 7] + dataColumns: t:tinyint, si:smallint, i:int, b:bigint, f:float, d:double, bo:boolean, s:string, ts:timestamp, dec:decimal(4,2), bin:binary + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: tinyint), KEY.reducesinkkey1 (type: float), KEY.reducesinkkey0 (type: double), VALUE._col5 (type: string) + outputColumnNames: _col0, _col4, _col5, _col7 + Statistics: Num rows: 8771 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: tinyint, _col4: float, _col5: double, _col7: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col4 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: last_value_window_0 + arguments: _col0 + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 8771 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col7 (type: string), last_value_window_0 (type: tinyint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 8771 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 11600 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 11600 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 100 + Processor Tree: + ListSink + +PREHOOK: query: select s, last_value(t) over (partition by d order by f) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select s, last_value(t) over (partition by d order by f) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +s last_value_window_0 +calvin miller 99 +holly polk 83 +wendy quirinius -3 +yuri laertes 115 +nick steinbeck 21 +tom young 34 +katie brown 66 +priscilla quirinius 67 +gabriella quirinius 37 +katie falkner 118 +xavier robinson 47 +ethan carson 105 +victor johnson 19 +jessica white 29 +zach white 99 +jessica king -3 +victor davidson 83 +holly falkner 110 +quinn falkner 33 +holly young 82 +nick robinson 106 +xavier steinbeck 8 +irene king 66 +quinn zipper 14 +priscilla miller 61 +yuri miller 7 +wendy zipper 1 +zach steinbeck 124 +fred nixon 91 +katie brown 104 +nick davidson 74 +gabriella davidson 112 +zach carson 109 +wendy king 78 +jessica quirinius 22 +holly hernandez 35 +tom xylophone 30 +quinn johnson 117 +yuri zipper -1 +gabriella brown 92 +david robinson 103 +mike nixon 96 +gabriella white 6 +rachel davidson 7 +yuri garcia 121 +yuri zipper 92 +jessica steinbeck 86 +katie hernandez 57 +alice king 26 +victor steinbeck 87 +katie ovid 4 +priscilla young 71 +quinn van buren 18 +quinn davidson 93 +gabriella brown -3 +zach laertes 124 +jessica ichabod 86 +irene falkner 49 +ethan miller 72 +irene carson 26 +priscilla zipper 37 +tom robinson 33 +sarah davidson 86 +katie polk 62 +nick white 8 +xavier laertes 17 +alice ichabod 18 +nick polk 52 +luke brown 47 +wendy allen 16 +gabriella robinson 6 +calvin ichabod -3 +holly steinbeck 25 +gabriella carson 39 +holly van buren 19 +tom nixon 45 +katie laertes -1 +mike brown 3 +oscar nixon 65 +zach garcia 29 +tom polk 93 +mike allen 115 +alice johnson 17 +holly robinson 43 +yuri young 24 +priscilla thompson 9 +rachel carson 24 +victor brown 34 +gabriella laertes 18 +bob carson 6 +holly allen 63 +fred nixon 111 +rachel carson 89 +alice nixon 48 +priscilla brown 68 +victor falkner 81 +tom white 43 +holly hernandez 4 +david garcia 101 +rachel ellison 51 +PREHOOK: query: explain vectorization detail +select s, first_value(s) over (partition by bo order by s) from over10k limit 100 +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select s, first_value(s) over (partition by bo order by s) from over10k limit 100 +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: over10k + Statistics: Num rows: 9784 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + Reduce Output Operator + key expressions: bo (type: boolean), s (type: string) + sort order: ++ + Map-reduce partition columns: bo (type: boolean) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [6, 7] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [6] + valueColumns: [] + Statistics: Num rows: 9784 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 11 + includeColumns: [6, 7] + dataColumns: t:tinyint, si:smallint, i:int, b:bigint, f:float, d:double, bo:boolean, s:string, ts:timestamp, dec:decimal(4,2), bin:binary + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: boolean), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col6, _col7 + Statistics: Num rows: 9784 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col6: boolean, _col7: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col7 ASC NULLS FIRST + partition by: _col6 + raw input shape: + window functions: + window function definition + alias: first_value_window_0 + arguments: _col7 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 9784 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col7 (type: string), first_value_window_0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9784 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 10400 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 10400 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: 100 + Processor Tree: + ListSink + +PREHOOK: query: select s, first_value(s) over (partition by bo order by s) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select s, first_value(s) over (partition by bo order by s) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +s first_value_window_0 +alice allen alice allen +alice allen alice allen +alice allen alice allen +alice allen alice allen +alice brown alice allen +alice brown alice allen +alice brown alice allen +alice brown alice allen +alice brown alice allen +alice brown alice allen +alice brown alice allen +alice brown alice allen +alice carson alice allen +alice carson alice allen +alice carson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice davidson alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice ellison alice allen +alice falkner alice allen +alice falkner alice allen +alice falkner alice allen +alice falkner alice allen +alice falkner alice allen +alice falkner alice allen +alice garcia alice allen +alice garcia alice allen +alice garcia alice allen +alice garcia alice allen +alice garcia alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice hernandez alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice ichabod alice allen +alice johnson alice allen +alice johnson alice allen +alice johnson alice allen +alice johnson alice allen +alice johnson alice allen +alice johnson alice allen +alice johnson alice allen +alice johnson alice allen +alice king alice allen +alice king alice allen +alice king alice allen +alice king alice allen +alice king alice allen +alice king alice allen +alice laertes alice allen +alice laertes alice allen +alice laertes alice allen +alice laertes alice allen +alice laertes alice allen +alice laertes alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice miller alice allen +alice nixon alice allen +alice nixon alice allen +alice nixon alice allen +PREHOOK: query: explain vectorization detail +select t, s, i, last_value(i) over (partition by t order by s) +from over10k where (s = 'oscar allen' or s = 'oscar carson') and t = 10 +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select t, s, i, last_value(i) over (partition by t order by s) +from over10k where (s = 'oscar allen' or s = 'oscar carson') and t = 10 +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: over10k + Statistics: Num rows: 9421 Data size: 1017544 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + Filter Operator + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: FilterExprAndExpr(children: FilterExprOrExpr(children: FilterStringGroupColEqualStringScalar(col 7, val oscar allen) -> boolean, FilterStringGroupColEqualStringScalar(col 7, val oscar carson) -> boolean) -> boolean, FilterLongColEqualLongScalar(col 0, val 10) -> boolean) -> boolean + predicate: (((s = 'oscar allen') or (s = 'oscar carson')) and (t = 10)) (type: boolean) + Statistics: Num rows: 4710 Data size: 508717 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: s (type: string) + sort order: + + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [7] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [2] + Statistics: Num rows: 4710 Data size: 508717 Basic stats: COMPLETE Column stats: NONE + value expressions: i (type: int) + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 11 + includeColumns: [0, 2, 7] + dataColumns: t:tinyint, si:smallint, i:int, b:bigint, f:float, d:double, bo:boolean, s:string, ts:timestamp, dec:decimal(4,2), bin:binary + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: VALUE._col2 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2, _col7 + Statistics: Num rows: 4710 Data size: 508717 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: int, _col7: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col7 ASC NULLS FIRST + partition by: UDFToByte(10) + raw input shape: + window functions: + window function definition + alias: last_value_window_0 + arguments: _col2 + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(MAX)~CURRENT + Statistics: Num rows: 4710 Data size: 508717 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: 10 (type: tinyint), _col7 (type: string), _col2 (type: int), last_value_window_0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 4710 Data size: 508717 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4710 Data size: 508717 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table if exists wtest +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table if exists wtest +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table wtest as +select a, b +from +( +SELECT explode( + map( + 3, array(1,2,3,4,5), + 1, array(int(null),int(null),int(null), int(null), int(null)), + 2, array(1,null,2, null, 3) + ) + ) as (a,barr) FROM (select * from src limit 1) s + ) s1 lateral view explode(barr) arr as b +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@src +PREHOOK: Output: database:default +PREHOOK: Output: default@wtest +POSTHOOK: query: create table wtest as +select a, b +from +( +SELECT explode( + map( + 3, array(1,2,3,4,5), + 1, array(int(null),int(null),int(null), int(null), int(null)), + 2, array(1,null,2, null, 3) + ) + ) as (a,barr) FROM (select * from src limit 1) s + ) s1 lateral view explode(barr) arr as b +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@src +POSTHOOK: Output: database:default +POSTHOOK: Output: default@wtest +POSTHOOK: Lineage: wtest.a SCRIPT [] +POSTHOOK: Lineage: wtest.b SCRIPT [] +a b +PREHOOK: query: explain vectorization detail +select a, b, +first_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select a, b, +first_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: wtest + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1] + Reduce Output Operator + key expressions: a (type: int), b (type: int) + sort order: ++ + Map-reduce partition columns: a (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [] + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: a:int, b:int + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: int, _col1: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: first_value_window_0 + arguments: _col1 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: first_value_window_1 + arguments: _col1, true + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: first_value_window_2 + arguments: _col1 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + window function definition + alias: first_value_window_3 + arguments: _col1, true + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), first_value_window_0 (type: int), first_value_window_1 (type: int), first_value_window_2 (type: int), first_value_window_3 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a, b, +first_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +PREHOOK: Input: default@wtest +#### A masked pattern was here #### +POSTHOOK: query: select a, b, +first_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wtest +#### A masked pattern was here #### +a b first_value_window_0 first_value_window_1 first_value_window_2 first_value_window_3 +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +2 NULL NULL NULL NULL NULL +2 NULL NULL 1 NULL 1 +2 1 NULL 1 NULL 1 +2 2 1 1 NULL 1 +2 3 2 2 NULL 1 +3 1 1 1 1 1 +3 2 1 1 1 1 +3 3 2 2 1 1 +3 4 3 3 1 1 +3 5 4 4 1 1 +PREHOOK: query: explain vectorization detail +select a, b, +first_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select a, b, +first_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: wtest + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1] + Reduce Output Operator + key expressions: a (type: int), b (type: int) + sort order: +- + Map-reduce partition columns: a (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [] + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: a:int, b:int + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: int, _col1: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 DESC NULLS LAST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: first_value_window_0 + arguments: _col1 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: first_value_window_1 + arguments: _col1, true + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: first_value_window_2 + arguments: _col1 + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + window function definition + alias: first_value_window_3 + arguments: _col1, true + name: first_value + window function: GenericUDAFFirstValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), first_value_window_0 (type: int), first_value_window_1 (type: int), first_value_window_2 (type: int), first_value_window_3 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a, b, +first_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +PREHOOK: Input: default@wtest +#### A masked pattern was here #### +POSTHOOK: query: select a, b, +first_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +first_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +first_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wtest +#### A masked pattern was here #### +a b first_value_window_0 first_value_window_1 first_value_window_2 first_value_window_3 +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +2 3 3 3 3 3 +2 2 3 3 3 3 +2 1 2 2 3 3 +2 NULL 1 1 3 3 +2 NULL NULL NULL 3 3 +3 5 5 5 5 5 +3 4 5 5 5 5 +3 3 4 4 5 5 +3 2 3 3 5 5 +3 1 2 2 5 5 +PREHOOK: query: explain vectorization detail +select a, b, +last_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select a, b, +last_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: wtest + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1] + Reduce Output Operator + key expressions: a (type: int), b (type: int) + sort order: ++ + Map-reduce partition columns: a (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [] + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: a:int, b:int + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: int, _col1: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: last_value_window_0 + arguments: _col1 + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: last_value_window_1 + arguments: _col1, true + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: last_value_window_2 + arguments: _col1 + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + window function definition + alias: last_value_window_3 + arguments: _col1, true + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), last_value_window_0 (type: int), last_value_window_1 (type: int), last_value_window_2 (type: int), last_value_window_3 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a, b, +last_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +PREHOOK: Input: default@wtest +#### A masked pattern was here #### +POSTHOOK: query: select a, b, +last_value(b) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wtest +#### A masked pattern was here #### +a b last_value_window_0 last_value_window_1 last_value_window_2 last_value_window_3 +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +2 NULL NULL NULL NULL NULL +2 NULL 1 1 1 1 +2 1 2 2 2 2 +2 2 3 3 3 3 +2 3 3 3 3 3 +3 1 2 2 2 2 +3 2 3 3 3 3 +3 3 4 4 4 4 +3 4 5 5 5 5 +3 5 5 5 5 5 +PREHOOK: query: explain vectorization detail +select a, b, +last_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +POSTHOOK: query: explain vectorization detail +select a, b, +last_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +Explain +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: wtest + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0, 1] + Reduce Output Operator + key expressions: a (type: int), b (type: int) + sort order: +- + Map-reduce partition columns: a (type: int) + Reduce Sink Vectorization: + className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [] + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + groupByVectorOutput: true + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: a:int, b:int + partitionColumnCount: 0 + Reducer 2 + Execution mode: llap + Reduce Vectorization: + enabled: true + enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + notVectorizedReason: PTF Operator (PTF) not supported + vectorized: false + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: int, _col1: int + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 DESC NULLS LAST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: last_value_window_0 + arguments: _col1 + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: last_value_window_1 + arguments: _col1, true + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(1)~FOLLOWING(1) + window function definition + alias: last_value_window_2 + arguments: _col1 + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + window function definition + alias: last_value_window_3 + arguments: _col1, true + name: last_value + window function: GenericUDAFLastValueEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(1) + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), last_value_window_0 (type: int), last_value_window_1 (type: int), last_value_window_2 (type: int), last_value_window_3 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 52 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a, b, +last_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +PREHOOK: type: QUERY +PREHOOK: Input: default@wtest +#### A masked pattern was here #### +POSTHOOK: query: select a, b, +last_value(b) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between 1 preceding and 1 following ) , +last_value(b) over (partition by a order by b desc rows between unbounded preceding and 1 following ) , +last_value(b, true) over (partition by a order by b desc rows between unbounded preceding and 1 following ) +from wtest +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wtest +#### A masked pattern was here #### +a b last_value_window_0 last_value_window_1 last_value_window_2 last_value_window_3 +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +1 NULL NULL NULL NULL NULL +2 3 2 2 2 2 +2 2 1 1 1 1 +2 1 NULL 1 NULL 1 +2 NULL NULL 1 NULL 1 +2 NULL NULL NULL NULL 1 +3 5 4 4 4 4 +3 4 3 3 3 3 +3 3 2 2 2 2 +3 2 1 1 1 1 +3 1 1 1 1 1 diff --git ql/src/test/results/clientpositive/llap/vectorization_limit.q.out ql/src/test/results/clientpositive/llap/vectorization_limit.q.out index c38a215..f341e01 100644 --- ql/src/test/results/clientpositive/llap/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_limit.q.out @@ -64,17 +64,17 @@ POSTHOOK: query: SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdoubl POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --1887561756 1839.0 -1887561756 -10011.0 -1887561756 -13877.0 --1887561756 10361.0 --1887561756 -8881.0 -1887561756 -2281.0 +-1887561756 -8881.0 +-1887561756 10361.0 +-1887561756 1839.0 -1887561756 9531.0 -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -137,15 +137,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 1, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 3 + dataColumns: KEY.reducesinkkey0:tinyint, KEY.reducesinkkey1:double, VALUE._col0:smallint + partitionColumnCount: 0 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: tinyint), KEY.reducesinkkey1 (type: double), VALUE._col0 (type: smallint) @@ -186,10 +197,17 @@ POSTHOOK: query: select ctinyint,cdouble,csmallint from alltypesorc where ctinyi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --64 -15920.0 -15920 -64 -10462.0 -10462 --64 -9842.0 -9842 --64 -8080.0 -8080 +-64 -15920.0 -15920 +-64 -1600.0 -1600 +-64 -200.0 -200 +-64 -2919.0 -2919 +-64 -3097.0 -3097 +-64 -3586.0 -3586 +-64 -4018.0 -4018 +-64 -4040.0 -4040 +-64 -4803.0 -4803 +-64 -6907.0 -6907 -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 @@ -197,19 +215,12 @@ POSTHOOK: Input: default@alltypesorc -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 --64 -6907.0 -6907 --64 -4803.0 -4803 --64 -4040.0 -4040 --64 -4018.0 -4018 --64 -3586.0 -3586 --64 -3097.0 -3097 --64 -2919.0 -2919 --64 -1600.0 -1600 --64 -200.0 -200 -PREHOOK: query: explain vectorization expression +-64 -8080.0 -8080 +-64 -9842.0 -9842 +PREHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -276,6 +287,12 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 + scratchColumnTypeNames: double Reducer 2 Execution mode: llap Reduce Vectorization: @@ -315,30 +332,30 @@ POSTHOOK: query: select ctinyint,avg(cdouble + 1) from alltypesorc group by ctin POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 9370.0945309795 --64 373.52941176470586 --63 2178.7272727272725 --62 245.69387755102042 --61 914.3404255319149 --60 1071.82 --59 318.27272727272725 --58 3483.2444444444445 --57 1867.0535714285713 --56 2595.818181818182 --55 2385.595744680851 --54 2712.7272727272725 --53 -532.7567567567568 --52 2810.705882352941 --51 -96.46341463414635 --50 -960.0192307692307 --49 768.7659574468086 --48 1672.909090909091 --47 -574.6428571428571 -46 3033.55 -PREHOOK: query: explain vectorization expression +-47 -574.6428571428571 +-48 1672.909090909091 +-49 768.7659574468086 +-50 -960.0192307692307 +-51 -96.46341463414635 +-52 2810.705882352941 +-53 -532.7567567567568 +-54 2712.7272727272725 +-55 2385.595744680851 +-56 2595.818181818182 +-57 1867.0535714285713 +-58 3483.2444444444445 +-59 318.27272727272725 +-60 1071.82 +-61 914.3404255319149 +-62 245.69387755102042 +-63 2178.7272727272725 +-64 373.52941176470586 +NULL 9370.0945309795 +PREHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -405,15 +422,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: a + reduceColumnSortOrder: + groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 1 + dataColumns: KEY._col0:tinyint + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator Group By Vectorization: @@ -457,30 +485,30 @@ POSTHOOK: query: select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL --64 --63 --62 --61 --60 --59 --58 --57 --56 --55 --54 --53 --52 --51 --50 --49 --48 --47 -46 -PREHOOK: query: explain vectorization expression +-47 +-48 +-49 +-50 +-51 +-52 +-53 +-54 +-55 +-56 +-57 +-58 +-59 +-60 +-61 +-62 +-63 +-64 +NULL +PREHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -532,8 +560,11 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: tinyint) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [] Statistics: Num rows: 6144 Data size: 55052 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -545,15 +576,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY._col0:tinyint, KEY._col1:double + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator Group By Vectorization: @@ -610,30 +652,30 @@ POSTHOOK: query: select ctinyint, count(distinct(cdouble)) from alltypesorc grou POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 2932 --64 24 --63 19 --62 27 --61 25 --60 27 --59 31 --58 23 --57 35 --56 36 --55 29 --54 26 --53 22 --52 33 --51 21 --50 30 --49 26 --48 29 --47 22 -46 24 -PREHOOK: query: explain vectorization expression +-47 22 +-48 29 +-49 26 +-50 30 +-51 21 +-52 33 +-53 22 +-54 26 +-55 29 +-56 36 +-57 35 +-58 23 +-59 31 +-60 27 +-61 25 +-62 27 +-63 19 +-64 24 +NULL 2932 +PREHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -658,10 +700,10 @@ POSTHOOK: query: select ctinyint,cdouble from alltypesorc order by ctinyint limi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -696,37 +738,31 @@ STAGE PLANS: predicateExpression: SelectColumnIsNotNull(col 0) -> boolean predicate: ctinyint is not null (type: boolean) Statistics: Num rows: 9173 Data size: 82188 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: cdouble (type: double), ctinyint (type: tinyint) - outputColumnNames: cdouble, ctinyint - Select Vectorization: - className: VectorSelectOperator - native: true - projectedOutputColumns: [5, 0] - Statistics: Num rows: 9173 Data size: 82188 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: sum(ctinyint) - Group By Vectorization: - aggregators: VectorUDAFSumLong(col 0) -> bigint - className: VectorGroupByOperator - vectorOutput: true - keyExpressions: col 5 - native: false - projectedOutputColumns: [0] - keys: cdouble (type: double) - mode: hash - outputColumnNames: _col0, _col1 + Group By Operator + aggregations: sum(ctinyint) + Group By Vectorization: + aggregators: VectorUDAFSumLong(col 0) -> bigint + className: VectorGroupByOperator + vectorOutput: true + keyExpressions: col 5 + native: false + projectedOutputColumns: [0] + keys: cdouble (type: double) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: [0] + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1] Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double) - sort order: + - Map-reduce partition columns: _col0 (type: double) - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint) + value expressions: _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -737,15 +773,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: a + reduceColumnSortOrder: + groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY._col0:double, VALUE._col0:bigint + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator aggregations: sum(VALUE._col0) @@ -775,10 +822,16 @@ STAGE PLANS: Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY.reducesinkkey0:bigint, KEY.reducesinkkey1:double + partitionColumnCount: 0 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey1 (type: double), KEY.reducesinkkey0 (type: bigint) @@ -819,23 +872,23 @@ POSTHOOK: query: select cdouble, sum(ctinyint) as sum from alltypesorc where cti POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL -32768 +-10462.0 -64 +-1121.0 -89 +-11322.0 -101 +-11492.0 -78 +-15920.0 -64 +-4803.0 -64 +-6907.0 -64 -7196.0 -2009 +-8080.0 -64 +-8118.0 -80 +-9842.0 -64 +10496.0 -67 15601.0 -1733 -4811.0 -115 --11322.0 -101 --1121.0 -89 -7705.0 -88 3520.0 -86 --8118.0 -80 +4811.0 -115 5241.0 -80 --11492.0 -78 -9452.0 -76 557.0 -75 -10496.0 -67 --15920.0 -64 --10462.0 -64 --9842.0 -64 --8080.0 -64 --6907.0 -64 --4803.0 -64 +7705.0 -88 +9452.0 -76 +NULL -32768 diff --git ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out index df4b0d8..b56b800 100644 --- ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out @@ -155,8 +155,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -371,8 +374,10 @@ STAGE PLANS: Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 2, 5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) Execution mode: vectorized, llap @@ -411,8 +416,10 @@ STAGE PLANS: Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -615,8 +622,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -769,8 +779,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -983,8 +996,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -1200,8 +1216,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -1419,8 +1438,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [0, 3, 4, 5, 6, 7, 8] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Execution mode: vectorized, llap @@ -1459,8 +1481,10 @@ STAGE PLANS: Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1633,8 +1657,10 @@ STAGE PLANS: Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1665,8 +1691,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [0, 3, 4, 5, 6, 7, 8] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Execution mode: vectorized, llap @@ -2244,8 +2273,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2457,8 +2489,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2740,8 +2775,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2957,8 +2995,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [0, 5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2997,8 +3038,10 @@ STAGE PLANS: Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: className: VectorReduceSinkLongOperator + keyColumns: [0] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -3241,8 +3284,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -3472,8 +3518,11 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [0, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [0] + valueColumns: [2] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: double) Execution mode: vectorized, llap @@ -3728,8 +3777,11 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkObjectHashOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + partitionColumns: [2] + valueColumns: [5, 7] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -4164,8 +4216,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [2] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_size (type: int) Execution mode: vectorized, llap @@ -4478,8 +4532,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: className: VectorReduceSinkStringOperator + keyColumns: [2] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [1, 5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_size (type: int) Execution mode: vectorized, llap @@ -4788,8 +4844,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -5074,8 +5132,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -5403,8 +5463,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -5701,8 +5763,10 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator + keyColumns: [2, 1] native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: [5] Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/windowing_navfn.q.out ql/src/test/results/clientpositive/windowing_navfn.q.out index 0976b54..3d783d0 100644 --- ql/src/test/results/clientpositive/windowing_navfn.q.out +++ ql/src/test/results/clientpositive/windowing_navfn.q.out @@ -44,6 +44,66 @@ POSTHOOK: query: load data local inpath '../../data/files/over10k' into table ov POSTHOOK: type: LOAD #### A masked pattern was here #### POSTHOOK: Output: default@over10k +PREHOOK: query: explain select row_number() over() from src where key = '238' +PREHOOK: type: QUERY +POSTHOOK: query: explain select row_number() over() from src where key = '238' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = '238') (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Operator Tree: + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: 0 ASC NULLS FIRST + partition by: 0 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: row_number_window_0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + PREHOOK: query: select row_number() over() from src where key = '238' PREHOOK: type: QUERY PREHOOK: Input: default@src