diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java index 94eaf56..defaf90 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java @@ -150,6 +150,25 @@ public void init(StructObjectInspector structObjectInspector, List proj } /* + * Initialize using an ObjectInspector array and a column projection array. + */ + public void init(TypeInfo[] typeInfos, int[] projectedColumns) + throws HiveException { + + final int count = typeInfos.length; + allocateArrays(count); + + for (int i = 0; i < count; i++) { + + int projectionColumnNum = projectedColumns[i]; + + TypeInfo typeInfo = typeInfos[i]; + + initEntry(i, projectionColumnNum, typeInfo); + } + } + + /* * Initialize using data type names. * No projection -- the column range 0 .. types.size()-1 */ 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 42ca4b7..fc5aea5 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 @@ -102,36 +102,29 @@ //--------------------------------------------------------------------------- // Whether there is to be a tag added to the end of each key and the tag value. - private transient boolean reduceSkipTag; - private transient byte reduceTagByte; + protected transient boolean reduceSkipTag; + protected transient byte reduceTagByte; // Binary sortable key serializer. protected transient BinarySortableSerializeWrite keyBinarySortableSerializeWrite; - // The serialized all null key and its hash code. - private transient byte[] nullBytes; - private transient int nullKeyHashCode; - // Lazy binary value serializer. - private transient LazyBinarySerializeWrite valueLazyBinarySerializeWrite; + protected transient LazyBinarySerializeWrite valueLazyBinarySerializeWrite; // This helper object serializes LazyBinary format reducer values from columns of a row // in a vectorized row batch. - private transient VectorSerializeRow valueVectorSerializeRow; + protected transient VectorSerializeRow valueVectorSerializeRow; // The output buffer used to serialize a value into. - private transient Output valueOutput; + protected transient Output valueOutput; // The hive key and bytes writable value needed to pass the key and value to the collector. - private transient HiveKey keyWritable; - private transient BytesWritable valueBytesWritable; + protected transient HiveKey keyWritable; + protected transient BytesWritable valueBytesWritable; // Where to write our key and value pairs. private transient OutputCollector out; - // The object that determines equal key series. - protected transient VectorKeySeriesSerialized serializedKeySeries; - private transient long numRows = 0; private transient long cntr = 1; private transient long logEveryNRows = 0; @@ -158,6 +151,8 @@ public VectorReduceSinkCommonOperator(CompilationOpContext ctx, VectorizationContext vContext, OperatorDesc conf) throws HiveException { this(ctx); + LOG.info("VectorReduceSinkCommonOperator constructor"); + ReduceSinkDesc desc = (ReduceSinkDesc) conf; this.conf = desc; vectorDesc = (VectorReduceSinkDesc) desc.getVectorDesc(); @@ -247,6 +242,46 @@ public VectorReduceSinkCommonOperator(CompilationOpContext ctx, protected void initializeOp(Configuration hconf) throws HiveException { super.initializeOp(hconf); + if (isLogDebugEnabled) { + LOG.debug("useUniformHash " + vectorReduceSinkInfo.getUseUniformHash()); + + LOG.debug("reduceSinkKeyColumnMap " + + (vectorReduceSinkInfo.getReduceSinkKeyColumnMap() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkKeyColumnMap()))); + LOG.debug("reduceSinkKeyTypeInfos " + + (vectorReduceSinkInfo.getReduceSinkKeyTypeInfos() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkKeyTypeInfos()))); + LOG.debug("reduceSinkKeyColumnVectorTypes " + + (vectorReduceSinkInfo.getReduceSinkKeyColumnVectorTypes() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkKeyColumnVectorTypes()))); + LOG.debug("reduceSinkKeyExpressions " + + (vectorReduceSinkInfo.getReduceSinkKeyExpressions() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkKeyExpressions()))); + + LOG.debug("reduceSinkValueColumnMap " + + (vectorReduceSinkInfo.getReduceSinkValueColumnMap() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkValueColumnMap()))); + LOG.debug("reduceSinkValueTypeInfos " + + (vectorReduceSinkInfo.getReduceSinkValueTypeInfos() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkValueTypeInfos()))); + LOG.debug("reduceSinkValueColumnVectorTypes " + + (vectorReduceSinkInfo.getReduceSinkValueColumnVectorTypes() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkValueColumnVectorTypes()))); + LOG.debug("reduceSinkValueExpressions " + + (vectorReduceSinkInfo.getReduceSinkValueExpressions() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkValueExpressions()))); + + LOG.debug("reduceSinkBucketColumnMap " + + (vectorReduceSinkInfo.getReduceSinkBucketColumnMap() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkBucketColumnMap()))); + LOG.debug("reduceSinkBucketTypeInfos " + + (vectorReduceSinkInfo.getReduceSinkBucketTypeInfos() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkBucketTypeInfos()))); + LOG.debug("reduceSinkBucketColumnVectorTypes " + + (vectorReduceSinkInfo.getReduceSinkBucketColumnVectorTypes() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkBucketColumnVectorTypes()))); + LOG.debug("reduceSinkBucketExpressions " + + (vectorReduceSinkInfo.getReduceSinkBucketExpressions() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkBucketExpressions()))); + + LOG.debug("reduceSinkPartitionColumnMap " + + (vectorReduceSinkInfo.getReduceSinkPartitionColumnMap() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkPartitionColumnMap()))); + LOG.debug("reduceSinkPartitionTypeInfos " + + (vectorReduceSinkInfo.getReduceSinkPartitionTypeInfos() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkPartitionTypeInfos()))); + LOG.debug("reduceSinkPartitionColumnVectorTypes " + + (vectorReduceSinkInfo.getReduceSinkPartitionColumnVectorTypes() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkPartitionColumnVectorTypes()))); + LOG.debug("reduceSinkPartitionExpressions " + + (vectorReduceSinkInfo.getReduceSinkPartitionExpressions() == null ? "NULL" : Arrays.toString(vectorReduceSinkInfo.getReduceSinkPartitionExpressions()))); + } + if (LOG.isDebugEnabled()) { // Determine the name of our map or reduce task for debug tracing. BaseWork work = Utilities.getMapWork(hconf); @@ -280,21 +315,6 @@ protected void initializeOp(Configuration hconf) throws HiveException { keyBinarySortableSerializeWrite = new BinarySortableSerializeWrite(columnSortOrder, columnNullMarker, columnNotNullMarker); - // Create all nulls key. - try { - Output nullKeyOutput = new Output(); - keyBinarySortableSerializeWrite.set(nullKeyOutput); - for (int i = 0; i < reduceSinkKeyColumnMap.length; i++) { - keyBinarySortableSerializeWrite.writeNull(); - } - int nullBytesLength = nullKeyOutput.getLength(); - nullBytes = new byte[nullBytesLength]; - System.arraycopy(nullKeyOutput.getData(), 0, nullBytes, 0, nullBytesLength); - nullKeyHashCode = HashCodeUtil.calculateBytesHashCode(nullBytes, 0, nullBytesLength); - } catch (Exception e) { - throw new HiveException(e); - } - valueLazyBinarySerializeWrite = new LazyBinarySerializeWrite(reduceSinkValueColumnMap.length); valueVectorSerializeRow = @@ -312,101 +332,6 @@ protected void initializeOp(Configuration hconf) throws HiveException { batchCounter = 0; } - @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; - } - - // Perform any key expressions. Results will go into scratch columns. - if (reduceSinkKeyExpressions != null) { - for (VectorExpression ve : reduceSinkKeyExpressions) { - ve.evaluate(batch); - } - } - - // Perform any value expressions. Results will go into scratch columns. - if (reduceSinkValueExpressions != null) { - for (VectorExpression ve : reduceSinkValueExpressions) { - ve.evaluate(batch); - } - } - - serializedKeySeries.processBatch(batch); - - boolean selectedInUse = batch.selectedInUse; - int[] selected = batch.selected; - - int keyLength; - int logical; - int end; - int batchIndex; - do { - if (serializedKeySeries.getCurrentIsAllNull()) { - - // Use the same logic as ReduceSinkOperator.toHiveKey. - // - if (tag == -1 || reduceSkipTag) { - keyWritable.set(nullBytes, 0, nullBytes.length); - } else { - keyWritable.setSize(nullBytes.length + 1); - System.arraycopy(nullBytes, 0, keyWritable.get(), 0, nullBytes.length); - keyWritable.get()[nullBytes.length] = reduceTagByte; - } - keyWritable.setDistKeyLength(nullBytes.length); - keyWritable.setHashCode(nullKeyHashCode); - - } else { - - // 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(); - if (tag == -1 || reduceSkipTag) { - keyWritable.set(serializedKeySeries.getSerializedBytes(), - serializedKeySeries.getSerializedStart(), keyLength); - } else { - keyWritable.setSize(keyLength + 1); - System.arraycopy(serializedKeySeries.getSerializedBytes(), - serializedKeySeries.getSerializedStart(), keyWritable.get(), 0, keyLength); - keyWritable.get()[keyLength] = reduceTagByte; - } - keyWritable.setDistKeyLength(keyLength); - keyWritable.setHashCode(serializedKeySeries.getCurrentHashCode()); - } - - logical = serializedKeySeries.getCurrentLogical(); - end = logical + serializedKeySeries.getCurrentDuplicateCount(); - do { - batchIndex = (selectedInUse ? selected[logical] : logical); - - valueLazyBinarySerializeWrite.reset(); - valueVectorSerializeRow.serializeWrite(batch, batchIndex); - - valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); - - collect(keyWritable, valueBytesWritable); - } while (++logical < end); - - if (!serializedKeySeries.next()) { - break; - } - } while (true); - - } catch (Exception e) { - throw new HiveException(e); - } - } - 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/VectorReduceSinkLongOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java index 325f773..0bc1cd1 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 @@ -32,7 +32,7 @@ /* * Specialized class for native vectorized reduce sink that is reducing on a single long key column. */ -public class VectorReduceSinkLongOperator extends VectorReduceSinkCommonOperator { +public class VectorReduceSinkLongOperator extends VectorReduceSinkUniformHashOperator { private static final long serialVersionUID = 1L; private static final String CLASS_NAME = VectorReduceSinkLongOperator.class.getName(); 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 2027187..1cca94d 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 @@ -32,7 +32,7 @@ * Specialized class for native vectorized reduce sink that is reducing on multiple key columns * (or a single non-long / non-string column). */ -public class VectorReduceSinkMultiKeyOperator extends VectorReduceSinkCommonOperator { +public class VectorReduceSinkMultiKeyOperator extends VectorReduceSinkUniformHashOperator { private static final long serialVersionUID = 1L; private static final String CLASS_NAME = VectorReduceSinkMultiKeyOperator.class.getName(); 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 new file mode 100644 index 0000000..6312c44 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java @@ -0,0 +1,289 @@ +/** + * 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 uniform hash (common) operator class for native vectorized reduce sink. + */ +public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOperator { + + private static final long serialVersionUID = 1L; + private static final String CLASS_NAME = VectorReduceSinkObjectHashOperator.class.getName(); + private static final Log LOG = LogFactory.getLog(CLASS_NAME); + + protected int[] reduceSinkBucketColumnMap; + protected TypeInfo[] reduceSinkBucketTypeInfos; + + protected VectorExpression[] reduceSinkBucketExpressions; + + protected int[] reduceSinkPartitionColumnMap; + protected TypeInfo[] reduceSinkPartitionTypeInfos; + + protected VectorExpression[] reduceSinkPartitionExpressions; + + // The above members are initialized by the constructor and must not be + // transient. + //--------------------------------------------------------------------------- + + protected transient Output keyOutput; + protected transient VectorSerializeRow keyVectorSerializeRow; + + private transient boolean hasBuckets; + private transient int numBuckets; + private transient ObjectInspector[] bucketObjectInspectors; + private transient VectorExtractRow bucketVectorExtractRow; + private transient Object[] bucketFieldValues; + + private transient boolean isPartitioned; + private transient ObjectInspector[] partitionObjectInspectors; + private transient VectorExtractRow partitionVectorExtractRow; + private transient Object[] partitionFieldValues; + private transient Random nonPartitionRandom; + + /** Kryo ctor. */ + protected VectorReduceSinkObjectHashOperator() { + super(); + } + + public VectorReduceSinkObjectHashOperator(CompilationOpContext ctx) { + super(ctx); + } + + public VectorReduceSinkObjectHashOperator(CompilationOpContext ctx, + VectorizationContext vContext, OperatorDesc conf) throws HiveException { + super(ctx, vContext, conf); + + LOG.info("VectorReduceSinkObjectHashOperator constructor vectorReduceSinkInfo " + vectorReduceSinkInfo); + + // This the is Object Hash class variation. + Preconditions.checkState(!vectorReduceSinkInfo.getUseUniformHash()); + + reduceSinkBucketColumnMap = vectorReduceSinkInfo.getReduceSinkBucketColumnMap(); + reduceSinkBucketTypeInfos = vectorReduceSinkInfo.getReduceSinkBucketTypeInfos(); + reduceSinkBucketExpressions = vectorReduceSinkInfo.getReduceSinkBucketExpressions(); + + reduceSinkPartitionColumnMap = vectorReduceSinkInfo.getReduceSinkPartitionColumnMap(); + reduceSinkPartitionTypeInfos = vectorReduceSinkInfo.getReduceSinkPartitionTypeInfos(); + reduceSinkPartitionExpressions = vectorReduceSinkInfo.getReduceSinkPartitionExpressions(); + } + + private ObjectInspector[] getObjectInspectorArray(TypeInfo[] typeInfos) { + final int size = typeInfos.length; + ObjectInspector[] objectInspectors = new ObjectInspector[size]; + for(int i = 0; i < size; i++) { + TypeInfo typeInfo = typeInfos[i]; + ObjectInspector standardWritableObjectInspector = + TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(typeInfo); + objectInspectors[i] = standardWritableObjectInspector; + } + return objectInspectors; + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + + keyOutput = new Output(); + keyBinarySortableSerializeWrite.set(keyOutput); + keyVectorSerializeRow = + new VectorSerializeRow( + keyBinarySortableSerializeWrite); + keyVectorSerializeRow.init(reduceSinkKeyTypeInfos, reduceSinkKeyColumnMap); + + hasBuckets = false; + isPartitioned = false; + numBuckets = 0; + + // Object Hash. + + numBuckets = conf.getNumBuckets(); + hasBuckets = (numBuckets > 0); + + if (hasBuckets) { + bucketObjectInspectors = getObjectInspectorArray(reduceSinkBucketTypeInfos); + bucketVectorExtractRow = new VectorExtractRow(); + bucketVectorExtractRow.init(reduceSinkBucketTypeInfos, reduceSinkBucketColumnMap); + bucketFieldValues = new Object[reduceSinkBucketTypeInfos.length]; + } + + isPartitioned = (conf.getPartitionCols() != null); + if (!isPartitioned) { + nonPartitionRandom = new Random(12345); + } else { + partitionObjectInspectors = getObjectInspectorArray(reduceSinkPartitionTypeInfos); + LOG.debug("*NEW* partitionObjectInspectors " + Arrays.toString(partitionObjectInspectors)); + partitionVectorExtractRow = new VectorExtractRow(); + partitionVectorExtractRow.init(reduceSinkPartitionTypeInfos, reduceSinkPartitionColumnMap); + partitionFieldValues = new Object[reduceSinkPartitionTypeInfos.length]; + } + } + + @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; + } + + // Perform any key expressions. Results will go into scratch columns. + if (reduceSinkKeyExpressions != null) { + for (VectorExpression ve : reduceSinkKeyExpressions) { + ve.evaluate(batch); + } + } + + // Perform any value expressions. Results will go into scratch columns. + if (reduceSinkValueExpressions != null) { + for (VectorExpression ve : reduceSinkValueExpressions) { + ve.evaluate(batch); + } + } + + // Perform any bucket expressions. Results will go into scratch columns. + if (reduceSinkBucketExpressions != null) { + for (VectorExpression ve : reduceSinkBucketExpressions) { + ve.evaluate(batch); + } + } + + // Perform any partition expressions. Results will go into scratch columns. + if (reduceSinkPartitionExpressions != null) { + for (VectorExpression ve : reduceSinkPartitionExpressions) { + ve.evaluate(batch); + } + } + + final boolean selectedInUse = batch.selectedInUse; + int[] selected = batch.selected; + + final int size = batch.size; + for (int logical = 0; logical < size; logical++) { + final int batchIndex = (selectedInUse ? selected[logical] : logical); + + final int hashCode; + if (!hasBuckets) { + if (!isPartitioned) { + hashCode = nonPartitionRandom.nextInt(); + } else { + partitionVectorExtractRow.extractRow(batch, batchIndex, partitionFieldValues); + hashCode = + ObjectInspectorUtils.getBucketHashCode( + partitionFieldValues, partitionObjectInspectors); + } + } else { + bucketVectorExtractRow.extractRow(batch, batchIndex, bucketFieldValues); + final int bucketNum = + ObjectInspectorUtils.getBucketNumber( + bucketFieldValues, bucketObjectInspectors, numBuckets); + if (!isPartitioned) { + hashCode = nonPartitionRandom.nextInt() * 31 + bucketNum; + } else { + partitionVectorExtractRow.extractRow(batch, batchIndex, partitionFieldValues); + hashCode = + ObjectInspectorUtils.getBucketHashCode( + partitionFieldValues, partitionObjectInspectors) * 31 + bucketNum; + } + } + + 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); + + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, batchIndex); + + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + + 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/VectorReduceSinkStringOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java index b655e6e..a838f4c 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 @@ -32,7 +32,7 @@ /* * Specialized class for native vectorized reduce sink that is reducing on a single long key column. */ -public class VectorReduceSinkStringOperator extends VectorReduceSinkCommonOperator { +public class VectorReduceSinkStringOperator extends VectorReduceSinkUniformHashOperator { private static final long serialVersionUID = 1L; private static final String CLASS_NAME = VectorReduceSinkStringOperator.class.getName(); 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 new file mode 100644 index 0000000..2dfa721 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java @@ -0,0 +1,218 @@ +/** + * 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; + +/** + * This class is uniform hash (common) operator class for native vectorized reduce sink. + */ +public abstract class VectorReduceSinkUniformHashOperator extends VectorReduceSinkCommonOperator { + + private static final long serialVersionUID = 1L; + private static final String CLASS_NAME = VectorReduceSinkUniformHashOperator.class.getName(); + private static final Log LOG = LogFactory.getLog(CLASS_NAME); + + // The above members are initialized by the constructor and must not be + // transient. + //--------------------------------------------------------------------------- + + // The serialized all null key and its hash code. + private transient byte[] nullBytes; + private transient int nullKeyHashCode; + + // The object that determines equal key series. + protected transient VectorKeySeriesSerialized serializedKeySeries; + + + /** Kryo ctor. */ + protected VectorReduceSinkUniformHashOperator() { + super(); + } + + public VectorReduceSinkUniformHashOperator(CompilationOpContext ctx) { + super(ctx); + } + + public VectorReduceSinkUniformHashOperator(CompilationOpContext ctx, + VectorizationContext vContext, OperatorDesc conf) throws HiveException { + super(ctx, vContext, conf); + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + + // Create all nulls key. + try { + Output nullKeyOutput = new Output(); + keyBinarySortableSerializeWrite.set(nullKeyOutput); + for (int i = 0; i < reduceSinkKeyColumnMap.length; i++) { + keyBinarySortableSerializeWrite.writeNull(); + } + int nullBytesLength = nullKeyOutput.getLength(); + nullBytes = new byte[nullBytesLength]; + System.arraycopy(nullKeyOutput.getData(), 0, nullBytes, 0, nullBytesLength); + nullKeyHashCode = HashCodeUtil.calculateBytesHashCode(nullBytes, 0, nullBytesLength); + } catch (Exception e) { + throw new HiveException(e); + } + } + + @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; + } + + // Perform any key expressions. Results will go into scratch columns. + if (reduceSinkKeyExpressions != null) { + for (VectorExpression ve : reduceSinkKeyExpressions) { + ve.evaluate(batch); + } + } + + // Perform any value expressions. Results will go into scratch columns. + if (reduceSinkValueExpressions != null) { + for (VectorExpression ve : reduceSinkValueExpressions) { + ve.evaluate(batch); + } + } + + serializedKeySeries.processBatch(batch); + + boolean selectedInUse = batch.selectedInUse; + int[] selected = batch.selected; + + int keyLength; + int logical; + int end; + int batchIndex; + do { + if (serializedKeySeries.getCurrentIsAllNull()) { + + // Use the same logic as ReduceSinkOperator.toHiveKey. + // + if (tag == -1 || reduceSkipTag) { + keyWritable.set(nullBytes, 0, nullBytes.length); + } else { + keyWritable.setSize(nullBytes.length + 1); + System.arraycopy(nullBytes, 0, keyWritable.get(), 0, nullBytes.length); + keyWritable.get()[nullBytes.length] = reduceTagByte; + } + keyWritable.setDistKeyLength(nullBytes.length); + keyWritable.setHashCode(nullKeyHashCode); + + } else { + + // 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(); + if (tag == -1 || reduceSkipTag) { + keyWritable.set(serializedKeySeries.getSerializedBytes(), + serializedKeySeries.getSerializedStart(), keyLength); + } else { + keyWritable.setSize(keyLength + 1); + System.arraycopy(serializedKeySeries.getSerializedBytes(), + serializedKeySeries.getSerializedStart(), keyWritable.get(), 0, keyLength); + keyWritable.get()[keyLength] = reduceTagByte; + } + keyWritable.setDistKeyLength(keyLength); + keyWritable.setHashCode(serializedKeySeries.getCurrentHashCode()); + } + + logical = serializedKeySeries.getCurrentLogical(); + end = logical + serializedKeySeries.getCurrentDuplicateCount(); + do { + batchIndex = (selectedInUse ? selected[logical] : logical); + + valueLazyBinarySerializeWrite.reset(); + valueVectorSerializeRow.serializeWrite(batch, batchIndex); + + valueBytesWritable.set(valueOutput.getData(), 0, valueOutput.getLength()); + + collect(keyWritable, valueBytesWritable); + } while (++logical < end); + + if (!serializedKeySeries.next()) { + break; + } + } while (true); + + } catch (Exception e) { + throw new HiveException(e); + } + } +} \ No newline at end of file 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 32ec1d7..bf60c10 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 @@ -29,6 +29,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Map.Entry; import java.util.Properties; import java.util.Set; @@ -62,6 +63,7 @@ import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinOuterStringOperator; 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; import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkStringOperator; import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor; import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type; @@ -143,6 +145,9 @@ import org.apache.hadoop.hive.ql.plan.VectorReduceSinkInfo; import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc; import org.apache.hadoop.hive.ql.plan.VectorSelectDesc; +import org.apache.hadoop.hive.ql.plan.VectorReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.VectorReduceSinkInfo; +import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc; import org.apache.hadoop.hive.ql.plan.api.OperatorType; import org.apache.hadoop.hive.ql.udf.UDFAcos; import org.apache.hadoop.hive.ql.udf.UDFAsin; @@ -2909,9 +2914,6 @@ private boolean canSpecializeMapJoin(Operator op, MapJoi Operator op, VectorizationContext vContext, ReduceSinkDesc desc, VectorReduceSinkInfo vectorReduceSinkInfo) throws HiveException { - Operator vectorOp = null; - Class> opClass = null; - Type[] reduceSinkKeyColumnVectorTypes = vectorReduceSinkInfo.getReduceSinkKeyColumnVectorTypes(); // By default, we can always use the multi-key class. @@ -2948,18 +2950,23 @@ private boolean canSpecializeMapJoin(Operator op, MapJoi } } - 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); + 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); + } + } else { + opClass = VectorReduceSinkObjectHashOperator.class; } VectorReduceSinkDesc vectorDesc = (VectorReduceSinkDesc) desc.getVectorDesc(); @@ -2967,9 +2974,17 @@ private boolean canSpecializeMapJoin(Operator op, MapJoi vectorDesc.setReduceSinkKeyType(reduceSinkKeyType); vectorDesc.setVectorReduceSinkInfo(vectorReduceSinkInfo); - vectorOp = OperatorFactory.getVectorOperator( - opClass, op.getCompilationOpContext(), op.getConf(), vContext); - LOG.info("Vectorizer vectorizeOperator reduce sink class " + vectorOp.getClass().getSimpleName()); + LOG.info("Vectorizer vectorizeOperator reduce sink class " + opClass.getSimpleName()); + + Operator vectorOp = null; + try { + vectorOp = OperatorFactory.getVectorOperator( + opClass, op.getCompilationOpContext(), op.getConf(), vContext); + } catch (Exception e) { + LOG.info("Vectorizer vectorizeOperator reduce sink class exception " + opClass.getSimpleName() + + " exception " + e); + throw new HiveException(e); + } return vectorOp; } @@ -2983,18 +2998,20 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, VectorReduceSinkDesc vectorDesc = new VectorReduceSinkDesc(); desc.setVectorDesc(vectorDesc); - boolean isVectorizationReduceSinkNativeEnabled = HiveConf.getBoolVar(hiveConf, - HiveConf.ConfVars.HIVE_VECTORIZATION_REDUCESINK_NEW_ENABLED); + // Various restrictions. - String engine = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); - - boolean hasBuckets = desc.getBucketCols() != null && !desc.getBucketCols().isEmpty(); + // Set this if we encounter a condition we were not expecting. + boolean isUnexpectedCondition = false; - boolean hasTopN = desc.getTopN() >= 0; + boolean isVectorizationReduceSinkNativeEnabled = + HiveConf.getBoolVar(hiveConf, + HiveConf.ConfVars.HIVE_VECTORIZATION_REDUCESINK_NEW_ENABLED); - boolean useUniformHash = desc.getReducerTraits().contains(UNIFORM); + String engine = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); + + boolean hasTopN = (desc.getTopN() >= 0); - boolean hasDistinctColumns = desc.getDistinctColumnIndices().size() > 0; + boolean hasDistinctColumns = (desc.getDistinctColumnIndices().size() > 0); TableDesc keyTableDesc = desc.getKeySerializeInfo(); Class keySerializerClass = keyTableDesc.getDeserializerClass(); @@ -3004,28 +3021,6 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, Class valueDeserializerClass = valueTableDesc.getDeserializerClass(); boolean isValueLazyBinary = (valueDeserializerClass == org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe.class); - // Remember the condition variables for EXPLAIN regardless. - vectorDesc.setIsVectorizationReduceSinkNativeEnabled(isVectorizationReduceSinkNativeEnabled); - vectorDesc.setEngine(engine); - vectorDesc.setHasBuckets(hasBuckets); - vectorDesc.setHasTopN(hasTopN); - vectorDesc.setUseUniformHash(useUniformHash); - vectorDesc.setHasDistinctColumns(hasDistinctColumns); - vectorDesc.setIsKeyBinarySortable(isKeyBinarySortable); - vectorDesc.setIsValueLazyBinary(isValueLazyBinary); - - // Many restrictions. - if (!isVectorizationReduceSinkNativeEnabled || - !isTezOrSpark || - hasBuckets || - hasTopN || - !useUniformHash || - hasDistinctColumns || - !isKeyBinarySortable || - !isValueLazyBinary) { - return false; - } - // We are doing work here we'd normally do in VectorGroupByCommonOperator's constructor. // So if we later decide not to specialize, we'll just waste any scratch columns allocated... @@ -3089,6 +3084,129 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, 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; + } + + } else { + + // Collect bucket and/or partition information for object hashing. + + int[] reduceSinkBucketColumnMap = null; + TypeInfo[] reduceSinkBucketTypeInfos = null; + Type[] reduceSinkBucketColumnVectorTypes = null; + VectorExpression[] reduceSinkBucketExpressions = null; + + List bucketDescs = desc.getBucketCols(); + if (bucketDescs != null) { + VectorExpression[] allBucketExpressions = vContext.getVectorExpressions(bucketDescs); + + reduceSinkBucketColumnMap = new int[bucketDescs.size()]; + reduceSinkBucketTypeInfos = new TypeInfo[bucketDescs.size()]; + reduceSinkBucketColumnVectorTypes = new Type[bucketDescs.size()]; + ArrayList reduceSinkBucketExpressionsList = new ArrayList(); + for (int i = 0; i < bucketDescs.size(); ++i) { + VectorExpression ve = allBucketExpressions[i]; + reduceSinkBucketColumnMap[i] = ve.getOutputColumn(); + reduceSinkBucketTypeInfos[i] = bucketDescs.get(i).getTypeInfo(); + reduceSinkBucketColumnVectorTypes[i] = + VectorizationContext.getColumnVectorTypeFromTypeInfo(reduceSinkBucketTypeInfos[i]); + if (!IdentityExpression.isColumnOnly(ve)) { + reduceSinkBucketExpressionsList.add(ve); + } + } + if (reduceSinkBucketExpressionsList.size() == 0) { + reduceSinkBucketExpressions = null; + } else { + reduceSinkBucketExpressions = reduceSinkBucketExpressionsList.toArray(new VectorExpression[0]); + } + } + + int[] reduceSinkPartitionColumnMap = null; + TypeInfo[] reduceSinkPartitionTypeInfos = null; + Type[] reduceSinkPartitionColumnVectorTypes = null; + VectorExpression[] reduceSinkPartitionExpressions = null; + + List partitionDescs = desc.getPartitionCols(); + if (partitionDescs != null) { + VectorExpression[] allPartitionExpressions = vContext.getVectorExpressions(partitionDescs); + + reduceSinkPartitionColumnMap = new int[partitionDescs.size()]; + reduceSinkPartitionTypeInfos = new TypeInfo[partitionDescs.size()]; + reduceSinkPartitionColumnVectorTypes = new Type[partitionDescs.size()]; + ArrayList reduceSinkPartitionExpressionsList = new ArrayList(); + for (int i = 0; i < partitionDescs.size(); ++i) { + VectorExpression ve = allPartitionExpressions[i]; + reduceSinkPartitionColumnMap[i] = ve.getOutputColumn(); + reduceSinkPartitionTypeInfos[i] = partitionDescs.get(i).getTypeInfo(); + reduceSinkPartitionColumnVectorTypes[i] = + VectorizationContext.getColumnVectorTypeFromTypeInfo(reduceSinkPartitionTypeInfos[i]); + if (!IdentityExpression.isColumnOnly(ve)) { + reduceSinkPartitionExpressionsList.add(ve); + } + } + if (reduceSinkPartitionExpressionsList.size() == 0) { + reduceSinkPartitionExpressions = null; + } else { + reduceSinkPartitionExpressions = reduceSinkPartitionExpressionsList.toArray(new VectorExpression[0]); + } + } + + vectorReduceSinkInfo.setReduceSinkBucketColumnMap(reduceSinkBucketColumnMap); + vectorReduceSinkInfo.setReduceSinkBucketTypeInfos(reduceSinkBucketTypeInfos); + vectorReduceSinkInfo.setReduceSinkBucketColumnVectorTypes(reduceSinkBucketColumnVectorTypes); + vectorReduceSinkInfo.setReduceSinkBucketExpressions(reduceSinkBucketExpressions); + + vectorReduceSinkInfo.setReduceSinkPartitionColumnMap(reduceSinkPartitionColumnMap); + vectorReduceSinkInfo.setReduceSinkPartitionTypeInfos(reduceSinkPartitionTypeInfos); + vectorReduceSinkInfo.setReduceSinkPartitionColumnVectorTypes(reduceSinkPartitionColumnVectorTypes); + vectorReduceSinkInfo.setReduceSinkPartitionExpressions(reduceSinkPartitionExpressions); + } + + // Remember the condition variables for EXPLAIN regardless. + vectorDesc.setIsVectorizationReduceSinkNativeEnabled(isVectorizationReduceSinkNativeEnabled); + vectorDesc.setEngine(engine); + vectorDesc.setHasTopN(hasTopN); + vectorDesc.setHasDistinctColumns(hasDistinctColumns); + vectorDesc.setIsKeyBinarySortable(isKeyBinarySortable); + vectorDesc.setIsValueLazyBinary(isValueLazyBinary); + + // This indicates we logged an inconsistency (from our point-of-view) and will not make this + // operator native... + vectorDesc.setIsUnexpectedCondition(isUnexpectedCondition); + + // Many restrictions. + if (!isVectorizationReduceSinkNativeEnabled || + !isTezOrSpark || + (useUniformHash && (hasEmptyBuckets || hasNoPartitions)) || + hasTopN || + hasDistinctColumns || + !isKeyBinarySortable || + !isValueLazyBinary || + isUnexpectedCondition) { + return false; + } + return true; } 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 11e9c20..38461d5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java @@ -534,15 +534,9 @@ public ReduceSinkOperatorExplainVectorization(ReduceSinkDesc reduceSinkDesc, Vec engineInSupported, engineInSupportedCondName), new VectorizationCondition( - !vectorReduceSinkDesc.getHasBuckets(), - "No buckets"), - new VectorizationCondition( !vectorReduceSinkDesc.getHasTopN(), "No TopN"), new VectorizationCondition( - vectorReduceSinkDesc.getUseUniformHash(), - "Uniform Hash"), - new VectorizationCondition( !vectorReduceSinkDesc.getHasDistinctColumns(), "No DISTINCT columns"), new VectorizationCondition( @@ -552,6 +546,15 @@ public ReduceSinkOperatorExplainVectorization(ReduceSinkDesc reduceSinkDesc, Vec vectorReduceSinkDesc.getIsValueLazyBinary(), "LazyBinarySerDe for values") }; + if (vectorReduceSinkDesc.getIsUnexpectedCondition()) { + VectorizationCondition[] newConditions = new VectorizationCondition[conditions.length + 1]; + System.arraycopy(conditions, 0, newConditions, 0, conditions.length); + newConditions[conditions.length] = + new VectorizationCondition( + false, + "NOT UnexpectedCondition"); + conditions = newConditions; + } return conditions; } 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 445dcca..d6230af 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java @@ -64,12 +64,11 @@ public VectorReduceSinkInfo getVectorReduceSinkInfo() { private boolean isVectorizationReduceSinkNativeEnabled; private String engine; - private boolean hasBuckets; private boolean hasTopN; - private boolean useUniformHash; private boolean hasDistinctColumns; private boolean isKeyBinarySortable; private boolean isValueLazyBinary; + private boolean isUnexpectedCondition; /* * The following conditions are for native Vector ReduceSink. @@ -86,24 +85,12 @@ public void setEngine(String engine) { public String getEngine() { return engine; } - public void setHasBuckets(boolean hasBuckets) { - this.hasBuckets = hasBuckets; - } - public boolean getHasBuckets() { - return hasBuckets; - } public void setHasTopN(boolean hasTopN) { this.hasTopN = hasTopN; } public boolean getHasTopN() { return hasTopN; } - public void setUseUniformHash(boolean useUniformHash) { - this.useUniformHash = useUniformHash; - } - public boolean getUseUniformHash() { - return useUniformHash; - } public void setHasDistinctColumns(boolean hasDistinctColumns) { this.hasDistinctColumns = hasDistinctColumns; } @@ -122,4 +109,10 @@ public void setIsValueLazyBinary(boolean isValueLazyBinary) { public boolean getIsValueLazyBinary() { return isValueLazyBinary; } + public void setIsUnexpectedCondition(boolean isUnexpectedCondition) { + this.isUnexpectedCondition = isUnexpectedCondition; + } + public boolean getIsUnexpectedCondition() { + return isUnexpectedCondition; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java index da6e606..5bafd5b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkInfo.java @@ -35,6 +35,8 @@ private static final long serialVersionUID = 1L; + private boolean useUniformHash; + private int[] reduceSinkKeyColumnMap; private TypeInfo[] reduceSinkKeyTypeInfos; private Type[] reduceSinkKeyColumnVectorTypes; @@ -45,7 +47,19 @@ private Type[] reduceSinkValueColumnVectorTypes; private VectorExpression[] reduceSinkValueExpressions; + private int[] reduceSinkBucketColumnMap; + private TypeInfo[] reduceSinkBucketTypeInfos; + private Type[] reduceSinkBucketColumnVectorTypes; + private VectorExpression[] reduceSinkBucketExpressions; + + private int[] reduceSinkPartitionColumnMap; + private TypeInfo[] reduceSinkPartitionTypeInfos; + private Type[] reduceSinkPartitionColumnVectorTypes; + private VectorExpression[] reduceSinkPartitionExpressions; + public VectorReduceSinkInfo() { + useUniformHash = false; + reduceSinkKeyColumnMap = null; reduceSinkKeyTypeInfos = null; reduceSinkKeyColumnVectorTypes = null; @@ -55,6 +69,24 @@ public VectorReduceSinkInfo() { reduceSinkValueTypeInfos = null; reduceSinkValueColumnVectorTypes = null; reduceSinkValueExpressions = null; + + reduceSinkBucketColumnMap = null; + reduceSinkBucketTypeInfos = null; + reduceSinkBucketColumnVectorTypes = null; + reduceSinkBucketExpressions = null; + + reduceSinkPartitionColumnMap = null; + reduceSinkPartitionTypeInfos = null; + reduceSinkPartitionColumnVectorTypes = null; + reduceSinkPartitionExpressions = null; + } + + public boolean getUseUniformHash() { + return useUniformHash; + } + + public void setUseUniformHash(boolean useUniformHash) { + this.useUniformHash = useUniformHash; } public int[] getReduceSinkKeyColumnMap() { @@ -120,4 +152,68 @@ public void setReduceSinkValueColumnVectorTypes(Type[] reduceSinkValueColumnVect public void setReduceSinkValueExpressions(VectorExpression[] reduceSinkValueExpressions) { this.reduceSinkValueExpressions = reduceSinkValueExpressions; } + + public int[] getReduceSinkBucketColumnMap() { + return reduceSinkBucketColumnMap; + } + + public void setReduceSinkBucketColumnMap(int[] reduceSinkBucketColumnMap) { + this.reduceSinkBucketColumnMap = reduceSinkBucketColumnMap; + } + + public TypeInfo[] getReduceSinkBucketTypeInfos() { + return reduceSinkBucketTypeInfos; + } + + public void setReduceSinkBucketTypeInfos(TypeInfo[] reduceSinkBucketTypeInfos) { + this.reduceSinkBucketTypeInfos = reduceSinkBucketTypeInfos; + } + + public Type[] getReduceSinkBucketColumnVectorTypes() { + return reduceSinkBucketColumnVectorTypes; + } + + public void setReduceSinkBucketColumnVectorTypes(Type[] reduceSinkBucketColumnVectorTypes) { + this.reduceSinkBucketColumnVectorTypes = reduceSinkBucketColumnVectorTypes; + } + + public VectorExpression[] getReduceSinkBucketExpressions() { + return reduceSinkBucketExpressions; + } + + public void setReduceSinkBucketExpressions(VectorExpression[] reduceSinkBucketExpressions) { + this.reduceSinkBucketExpressions = reduceSinkBucketExpressions; + } + + public int[] getReduceSinkPartitionColumnMap() { + return reduceSinkPartitionColumnMap; + } + + public void setReduceSinkPartitionColumnMap(int[] reduceSinkPartitionColumnMap) { + this.reduceSinkPartitionColumnMap = reduceSinkPartitionColumnMap; + } + + public TypeInfo[] getReduceSinkPartitionTypeInfos() { + return reduceSinkPartitionTypeInfos; + } + + public void setReduceSinkPartitionTypeInfos(TypeInfo[] reduceSinkPartitionTypeInfos) { + this.reduceSinkPartitionTypeInfos = reduceSinkPartitionTypeInfos; + } + + public Type[] getReduceSinkPartitionColumnVectorTypes() { + return reduceSinkPartitionColumnVectorTypes; + } + + public void setReduceSinkPartitionColumnVectorTypes(Type[] reduceSinkPartitionColumnVectorTypes) { + this.reduceSinkPartitionColumnVectorTypes = reduceSinkPartitionColumnVectorTypes; + } + + public VectorExpression[] getReduceSinkPartitionExpressions() { + return reduceSinkPartitionExpressions; + } + + public void setReduceSinkPartitionExpressions(VectorExpression[] reduceSinkPartitionExpressions) { + this.reduceSinkPartitionExpressions = reduceSinkPartitionExpressions; + } } diff --git ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out index 0e16ff1..85d65d6 100644 --- ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out +++ ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out @@ -1106,10 +1106,10 @@ Table: over1k_part_buck_orc #### A masked pattern was here #### Partition Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} - numFiles 8 + numFiles 2 numRows 32 rawDataSize 640 - totalSize 4524 + totalSize 1424 #### A masked pattern was here #### # Storage Information @@ -1147,10 +1147,10 @@ Table: over1k_part_buck_orc #### A masked pattern was here #### Partition Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} - numFiles 4 - numRows 6 - rawDataSize 120 - totalSize 2004 + numFiles 2 + numRows 4 + rawDataSize 80 + totalSize 936 #### A masked pattern was here #### # Storage Information @@ -1188,10 +1188,10 @@ Table: over1k_part_buck_sort_orc #### A masked pattern was here #### Partition Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} - numFiles 8 + numFiles 2 numRows 32 rawDataSize 640 - totalSize 4510 + totalSize 1416 #### A masked pattern was here #### # Storage Information @@ -1229,10 +1229,10 @@ Table: over1k_part_buck_sort_orc #### A masked pattern was here #### Partition Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} - numFiles 4 - numRows 6 - rawDataSize 120 - totalSize 2004 + numFiles 2 + numRows 4 + rawDataSize 80 + totalSize 944 #### A masked pattern was here #### # Storage Information @@ -1283,7 +1283,7 @@ POSTHOOK: Input: default@over1k_part_buck_orc POSTHOOK: Input: default@over1k_part_buck_orc@t=27 POSTHOOK: Input: default@over1k_part_buck_orc@t=__HIVE_DEFAULT_PARTITION__ #### A masked pattern was here #### -38 +34 PREHOOK: query: select count(*) from over1k_part_buck_sort_orc PREHOOK: type: QUERY PREHOOK: Input: default@over1k_part_buck_sort_orc @@ -1296,7 +1296,7 @@ POSTHOOK: Input: default@over1k_part_buck_sort_orc POSTHOOK: Input: default@over1k_part_buck_sort_orc@t=27 POSTHOOK: Input: default@over1k_part_buck_sort_orc@t=__HIVE_DEFAULT_PARTITION__ #### A masked pattern was here #### -38 +34 PREHOOK: query: create table over1k_part2_orc( si smallint, i int, @@ -2472,9 +2472,9 @@ Table: over1k_part_buck_sort2_orc Partition Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} numFiles 1 - numRows 3 - rawDataSize 78 - totalSize 81 + numRows 2 + rawDataSize 52 + totalSize 27 #### A masked pattern was here #### # Storage Information @@ -2534,8 +2534,6 @@ POSTHOOK: Input: default@over1k_part_buck_sort2_orc@t=__HIVE_DEFAULT_PARTITION__ 503 65628 4294967371 95.07 27 401 65779 4294967402 97.39 27 340 65677 4294967461 98.96 27 -409 65536 4294967490 46.97 NULL -374 65560 4294967516 65.43 NULL 473 65720 4294967324 80.74 NULL PREHOOK: query: explain select count(*) from over1k_part_buck_sort2_orc PREHOOK: type: QUERY @@ -2557,9 +2555,9 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k_part_buck_sort2_orc - Statistics: Num rows: 19 Data size: 645 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 18 Data size: 611 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - Statistics: Num rows: 19 Data size: 645 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 18 Data size: 611 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() mode: hash @@ -2605,4 +2603,4 @@ POSTHOOK: Input: default@over1k_part_buck_sort2_orc POSTHOOK: Input: default@over1k_part_buck_sort2_orc@t=27 POSTHOOK: Input: default@over1k_part_buck_sort2_orc@t=__HIVE_DEFAULT_PARTITION__ #### A masked pattern was here #### -19 +17 diff --git ql/src/test/results/clientpositive/llap/vector_adaptor_usage_mode.q.out ql/src/test/results/clientpositive/llap/vector_adaptor_usage_mode.q.out index 8b82442..5b17144 100644 --- ql/src/test/results/clientpositive/llap/vector_adaptor_usage_mode.q.out +++ ql/src/test/results/clientpositive/llap/vector_adaptor_usage_mode.q.out @@ -1101,7 +1101,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 5 Data size: 452 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_auto_smb_mapjoin_14.q.out ql/src/test/results/clientpositive/llap/vector_auto_smb_mapjoin_14.q.out index fbb2676..10bd85e 100644 --- ql/src/test/results/clientpositive/llap/vector_auto_smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/llap/vector_auto_smb_mapjoin_14.q.out @@ -285,10 +285,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 @@ -532,7 +531,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 5 Data size: 465 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -587,7 +586,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 5 Data size: 465 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) @@ -1293,7 +1292,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 10 Data size: 930 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1336,7 +1335,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 10 Data size: 930 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_between_columns.q.out ql/src/test/results/clientpositive/llap/vector_between_columns.q.out index 8897163..923e579 100644 --- ql/src/test/results/clientpositive/llap/vector_between_columns.q.out +++ ql/src/test/results/clientpositive/llap/vector_between_columns.q.out @@ -162,10 +162,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 5 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: smallint) Execution mode: vectorized, llap @@ -175,7 +174,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true @@ -330,10 +329,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 5 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: smallint) Execution mode: vectorized, llap @@ -343,7 +341,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true diff --git ql/src/test/results/clientpositive/llap/vector_between_in.q.out ql/src/test/results/clientpositive/llap/vector_between_in.q.out index aff55f2..9b58d47 100644 --- ql/src/test/results/clientpositive/llap/vector_between_in.q.out +++ ql/src/test/results/clientpositive/llap/vector_between_in.q.out @@ -59,10 +59,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -71,7 +70,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -164,10 +163,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -265,10 +263,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(20,10)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -277,7 +274,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -370,10 +367,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -471,10 +467,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1365 Data size: 274112 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -483,7 +478,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -568,10 +563,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 10923 Data size: 2193503 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -580,7 +574,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -665,10 +659,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(20,10)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1365 Data size: 274112 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -677,7 +670,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -770,10 +763,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -1127,7 +1119,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -1167,10 +1159,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1266,7 +1257,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -1306,10 +1297,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1405,7 +1395,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -1445,10 +1435,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1544,7 +1533,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -1584,10 +1573,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out index 3f7082b..63c3193 100644 --- ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out +++ ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out @@ -180,10 +180,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -226,7 +225,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)) Execution mode: vectorized, llap @@ -350,7 +349,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -568,7 +567,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: binary) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_char_2.q.out ql/src/test/results/clientpositive/llap/vector_char_2.q.out index 98db4c1..c330097 100644 --- ql/src/test/results/clientpositive/llap/vector_char_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_char_2.q.out @@ -113,7 +113,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -157,8 +157,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 250 Data size: 49500 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -311,7 +311,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -355,8 +355,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 250 Data size: 49500 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) diff --git ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out index c5b9287..e0df3c0 100644 --- ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out +++ ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out @@ -186,10 +186,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: char(10)) Execution mode: vectorized, llap @@ -199,7 +198,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -232,7 +231,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Execution mode: vectorized, llap @@ -344,7 +343,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Execution mode: vectorized, llap @@ -398,10 +397,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: char(20)) Execution mode: vectorized, llap @@ -411,7 +409,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 3 @@ -527,10 +525,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -540,7 +537,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -573,7 +570,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 273 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_char_simple.q.out ql/src/test/results/clientpositive/llap/vector_char_simple.q.out index a8e0071..d3ab3f2 100644 --- ql/src/test/results/clientpositive/llap/vector_char_simple.q.out +++ ql/src/test/results/clientpositive/llap/vector_char_simple.q.out @@ -253,8 +253,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: diff --git ql/src/test/results/clientpositive/llap/vector_coalesce.q.out ql/src/test/results/clientpositive/llap/vector_coalesce.q.out index 82fb03f..578f849 100644 --- ql/src/test/results/clientpositive/llap/vector_coalesce.q.out +++ ql/src/test/results/clientpositive/llap/vector_coalesce.q.out @@ -41,8 +41,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -147,8 +147,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -252,8 +252,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -358,8 +358,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -462,8 +462,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: diff --git ql/src/test/results/clientpositive/llap/vector_coalesce_2.q.out ql/src/test/results/clientpositive/llap/vector_coalesce_2.q.out index 5a283bc..c912b2e 100644 --- ql/src/test/results/clientpositive/llap/vector_coalesce_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_coalesce_2.q.out @@ -241,7 +241,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 4 Data size: 510 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_count.q.out ql/src/test/results/clientpositive/llap/vector_count.q.out index 1bda3d3..a6e2f59 100644 --- ql/src/test/results/clientpositive/llap/vector_count.q.out +++ ql/src/test/results/clientpositive/llap/vector_count.q.out @@ -99,8 +99,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE value expressions: _col5 (type: bigint) Execution mode: vectorized, llap @@ -208,8 +208,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE value expressions: _col4 (type: bigint), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col9 (type: bigint) Execution mode: vectorized, llap @@ -301,8 +301,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE value expressions: d (type: int) Execution mode: vectorized, llap @@ -397,8 +397,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out index 5709538..3c03787 100644 --- ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out +++ ql/src/test/results/clientpositive/llap/vector_count_distinct.q.out @@ -1282,7 +1282,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2000 Data size: 3520000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1329,10 +1329,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_data_types.q.out ql/src/test/results/clientpositive/llap/vector_data_types.q.out index 77e83cb..0ad6cfc 100644 --- ql/src/test/results/clientpositive/llap/vector_data_types.q.out +++ ql/src/test/results/clientpositive/llap/vector_data_types.q.out @@ -237,8 +237,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary) diff --git ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out index f514cf4..b4c4cc9 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out @@ -86,7 +86,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: bigint), _col6 (type: decimal(23,14)), _col7 (type: decimal(23,14)), _col8 (type: decimal(33,14)), _col9 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out index 6828396..02888e7 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out @@ -65,8 +65,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 455 Data size: 78809 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out index b89d277..9284a0a 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out @@ -183,7 +183,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_decimal_round.q.out ql/src/test/results/clientpositive/llap/vector_decimal_round.q.out index 2b1ad08..2a5e515 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_round.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_round.q.out @@ -467,10 +467,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(10,0)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(11,0)) Execution mode: vectorized, llap @@ -480,7 +479,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -569,10 +568,10 @@ STAGE PLANS: key expressions: round(_col0, -1) (type: decimal(11,0)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + keyExpressions: FuncRoundWithNumDigitsDecimalToDecimal(col 0, decimalPlaces -1) -> 1:decimal(11,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 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized, llap @@ -582,7 +581,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_decimal_round_2.q.out ql/src/test/results/clientpositive/llap/vector_decimal_round_2.q.out index 14719e0..89ca014 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_round_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_round_2.q.out @@ -75,10 +75,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(21,0)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(21,0)), _col2 (type: decimal(22,1)), _col3 (type: decimal(23,2)), _col4 (type: decimal(24,3)), _col5 (type: decimal(21,0)), _col6 (type: decimal(21,0)), _col7 (type: decimal(21,0)), _col8 (type: decimal(21,0)), _col9 (type: decimal(21,0)), _col10 (type: decimal(21,0)), _col11 (type: decimal(21,0)), _col12 (type: decimal(21,0)) Execution mode: vectorized, llap @@ -88,7 +87,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -227,10 +226,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(21,0)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(21,0)), _col2 (type: decimal(22,1)), _col3 (type: decimal(23,2)), _col4 (type: decimal(24,3)), _col5 (type: decimal(25,4)), _col6 (type: decimal(21,0)), _col7 (type: decimal(21,0)), _col8 (type: decimal(21,0)), _col9 (type: decimal(21,0)), _col10 (type: decimal(21,0)), _col11 (type: decimal(21,0)), _col12 (type: decimal(22,1)), _col13 (type: decimal(23,2)), _col14 (type: decimal(24,3)), _col15 (type: decimal(25,4)), _col16 (type: decimal(21,0)), _col17 (type: decimal(21,0)), _col18 (type: decimal(21,0)), _col19 (type: decimal(21,0)) Execution mode: vectorized, llap @@ -240,7 +238,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -406,10 +404,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(21,0)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(21,0)), _col2 (type: decimal(21,0)), _col3 (type: decimal(21,0)), _col4 (type: decimal(21,0)), _col5 (type: decimal(21,0)), _col6 (type: decimal(21,0)), _col7 (type: decimal(21,0)), _col8 (type: decimal(21,0)), _col9 (type: decimal(21,0)), _col10 (type: decimal(21,0)), _col11 (type: decimal(21,0)), _col12 (type: decimal(21,0)), _col13 (type: decimal(21,0)), _col14 (type: decimal(21,0)), _col15 (type: decimal(21,0)), _col16 (type: decimal(21,0)), _col17 (type: decimal(22,1)), _col18 (type: decimal(23,2)), _col19 (type: decimal(24,3)), _col20 (type: decimal(25,4)), _col21 (type: decimal(26,5)), _col22 (type: decimal(27,6)), _col23 (type: decimal(28,7)), _col24 (type: decimal(29,8)), _col25 (type: decimal(30,9)), _col26 (type: decimal(31,10)), _col27 (type: decimal(32,11)), _col28 (type: decimal(33,12)), _col29 (type: decimal(34,13)), _col31 (type: decimal(35,14)), _col32 (type: decimal(36,15)), _col33 (type: decimal(37,16)) Execution mode: vectorized, llap @@ -419,7 +416,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -574,10 +571,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(30,9)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(30,9)) Execution mode: vectorized, llap @@ -587,7 +583,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_distinct_2.q.out ql/src/test/results/clientpositive/llap/vector_distinct_2.q.out index a644f8b..84022f8 100644 --- ql/src/test/results/clientpositive/llap/vector_distinct_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_distinct_2.q.out @@ -157,7 +157,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_empty_where.q.out ql/src/test/results/clientpositive/llap/vector_empty_where.q.out index ed9d551..f582ca3 100644 --- ql/src/test/results/clientpositive/llap/vector_empty_where.q.out +++ ql/src/test/results/clientpositive/llap/vector_empty_where.q.out @@ -62,7 +62,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2888 Data size: 8628 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -109,10 +109,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reducer 3 @@ -218,7 +217,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2888 Data size: 8628 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -265,10 +264,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reducer 3 @@ -382,7 +380,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2888 Data size: 8628 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -429,10 +427,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reducer 3 @@ -546,7 +543,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2888 Data size: 8628 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -593,10 +590,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_groupby4.q.out ql/src/test/results/clientpositive/llap/vector_groupby4.q.out index 8c3949b..3b3b801 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby4.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby4.q.out @@ -67,10 +67,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: rand() (type: double) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -79,7 +78,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -110,7 +109,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_groupby6.q.out ql/src/test/results/clientpositive/llap/vector_groupby6.q.out index 6ea6ca8..7e21493 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby6.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby6.q.out @@ -67,10 +67,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: rand() (type: double) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -79,7 +78,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -110,7 +109,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_groupby_3.q.out ql/src/test/results/clientpositive/llap/vector_groupby_3.q.out index 454c298..064b197 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby_3.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby_3.q.out @@ -159,7 +159,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: bigint) Execution mode: vectorized, llap 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 fcf798e..9eb2747 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 @@ -102,7 +102,7 @@ STAGE PLANS: className: VectorReduceSinkMultiKeyOperator 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 40 Basic stats: COMPLETE Column stats: NONE value expressions: _col3 (type: bigint) Execution mode: vectorized, llap @@ -242,7 +242,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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_groupby_mapjoin.q.out ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out index bc7873b..af5a5ab 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out @@ -184,10 +184,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 6 @@ -227,7 +226,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 205 Data size: 18655 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) diff --git ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out index 65f94f0..80e4e9e 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out @@ -277,7 +277,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -448,7 +448,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -506,10 +506,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 250 Data size: 22069 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: vectorized, llap @@ -729,10 +728,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int) Execution mode: vectorized, llap @@ -793,10 +791,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 250 Data size: 22069 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: bigint) Reducer 3 @@ -1019,7 +1016,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int) Execution mode: vectorized, llap @@ -1080,10 +1077,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 250 Data size: 22069 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out index bb99556..8849db7 100644 --- ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out +++ ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out @@ -183,7 +183,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -313,7 +313,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_if_expr.q.out ql/src/test/results/clientpositive/llap/vector_if_expr.q.out index 4685e62..f06086e 100644 --- ql/src/test/results/clientpositive/llap/vector_if_expr.q.out +++ ql/src/test/results/clientpositive/llap/vector_if_expr.q.out @@ -48,10 +48,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 4587 Data size: 857712 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string) Execution mode: vectorized, llap @@ -61,7 +60,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_include_no_sel.q.out ql/src/test/results/clientpositive/llap/vector_include_no_sel.q.out index 133f8a8..029b3f5 100644 --- ql/src/test/results/clientpositive/llap/vector_include_no_sel.q.out +++ ql/src/test/results/clientpositive/llap/vector_include_no_sel.q.out @@ -203,8 +203,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.vectorized.execution.reducesink.new.enabled IS false, Uniform Hash IS false + nativeConditionsMet: 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 + nativeConditionsNotMet: hive.vectorized.execution.reducesink.new.enabled IS false Statistics: Num rows: 200 Data size: 74200 Basic stats: COMPLETE Column stats: NONE value expressions: cd_demo_sk (type: int), cd_marital_status (type: string) Execution mode: vectorized, llap @@ -268,8 +268,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.vectorized.execution.reducesink.new.enabled IS false, Uniform Hash IS false + nativeConditionsMet: 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 + nativeConditionsNotMet: hive.vectorized.execution.reducesink.new.enabled IS false Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (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 3058f0f..fc7b9a3 100644 --- ql/src/test/results/clientpositive/llap/vector_inner_join.q.out +++ ql/src/test/results/clientpositive/llap/vector_inner_join.q.out @@ -153,7 +153,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -317,7 +317,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -511,7 +511,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -599,7 +599,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -832,7 +832,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -996,7 +996,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1160,7 +1160,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1248,7 +1248,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 91 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1412,7 +1412,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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_interval_1.q.out ql/src/test/results/clientpositive/llap/vector_interval_1.q.out index 51ed381..debf5ab 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_1.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_1.q.out @@ -89,10 +89,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: interval_year_month), _col4 (type: interval_day_time) Execution mode: vectorized, llap @@ -102,7 +101,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -218,10 +217,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: interval_year_month), _col3 (type: interval_year_month), _col5 (type: interval_year_month), _col6 (type: interval_year_month) Execution mode: vectorized, llap @@ -231,7 +229,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -355,10 +353,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col5 (type: interval_day_time), _col6 (type: interval_day_time) Execution mode: vectorized, llap @@ -368,7 +365,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -504,10 +501,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: date), _col2 (type: date), _col3 (type: date), _col4 (type: date), _col5 (type: date), _col6 (type: date), _col7 (type: timestamp), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp) Execution mode: vectorized, llap @@ -517,7 +513,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -664,10 +660,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp), _col7 (type: timestamp), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp) Execution mode: vectorized, llap @@ -677,7 +672,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -806,10 +801,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time) Execution mode: vectorized, llap @@ -819,7 +813,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -930,10 +924,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time) Execution mode: vectorized, llap @@ -943,7 +936,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1060,10 +1053,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col4 (type: interval_day_time), _col5 (type: interval_day_time), _col6 (type: interval_day_time) Execution mode: vectorized, llap @@ -1073,7 +1065,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_interval_2.q.out ql/src/test/results/clientpositive/llap/vector_interval_2.q.out index bb09799..2669b12 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_2.q.out @@ -143,10 +143,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 788 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: boolean), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: boolean), _col13 (type: boolean), _col14 (type: boolean), _col15 (type: boolean), _col16 (type: boolean), _col17 (type: boolean), _col18 (type: boolean), _col19 (type: boolean), _col20 (type: boolean), _col21 (type: boolean), _col22 (type: boolean), _col23 (type: boolean), _col24 (type: boolean) Execution mode: vectorized, llap @@ -156,7 +155,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -351,10 +350,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 788 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: boolean), _col10 (type: boolean), _col11 (type: boolean), _col13 (type: boolean), _col14 (type: boolean), _col15 (type: boolean), _col16 (type: boolean), _col17 (type: boolean) Execution mode: vectorized, llap @@ -364,7 +362,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -559,10 +557,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 788 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: boolean), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: boolean), _col13 (type: boolean), _col14 (type: boolean), _col15 (type: boolean), _col16 (type: boolean), _col17 (type: boolean), _col18 (type: boolean), _col19 (type: boolean), _col20 (type: boolean), _col21 (type: boolean), _col22 (type: boolean), _col23 (type: boolean), _col24 (type: boolean) Execution mode: vectorized, llap @@ -572,7 +569,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -767,10 +764,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2 Data size: 788 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col7 (type: boolean), _col8 (type: boolean), _col9 (type: boolean), _col10 (type: boolean), _col11 (type: boolean), _col13 (type: boolean), _col14 (type: boolean), _col15 (type: boolean), _col16 (type: boolean), _col17 (type: boolean) Execution mode: vectorized, llap @@ -780,7 +776,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -967,10 +963,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 394 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -979,7 +974,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1163,10 +1158,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 394 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1175,7 +1169,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1349,10 +1343,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 394 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1361,7 +1354,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1535,10 +1528,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 394 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1547,7 +1539,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1731,10 +1723,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 394 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1743,7 +1734,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1927,10 +1918,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 394 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1939,7 +1929,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out index b746c68..aadb6e7 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out @@ -97,10 +97,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: date), _col2 (type: date), _col3 (type: date), _col4 (type: date), _col5 (type: date), _col6 (type: date) Execution mode: vectorized, llap @@ -110,7 +109,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -281,10 +280,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time) Execution mode: vectorized, llap @@ -294,7 +292,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -465,10 +463,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp) Execution mode: vectorized, llap @@ -478,7 +475,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -646,8 +643,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -785,10 +782,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp) Execution mode: vectorized, llap @@ -798,7 +794,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -971,10 +967,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col4 (type: interval_day_time) Execution mode: vectorized, llap @@ -984,7 +979,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1157,10 +1152,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp) Execution mode: vectorized, llap @@ -1170,7 +1164,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out index ee7b160..eee37d0 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out @@ -293,7 +293,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1000 Data size: 458448 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_join30.q.out ql/src/test/results/clientpositive/llap/vector_join30.q.out index 5b3beb0..6af0959 100644 --- ql/src/test/results/clientpositive/llap/vector_join30.q.out +++ ql/src/test/results/clientpositive/llap/vector_join30.q.out @@ -71,10 +71,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -83,7 +82,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 4 @@ -113,10 +112,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -126,7 +124,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -175,10 +173,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 @@ -237,7 +234,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) @@ -320,10 +317,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -332,7 +328,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 4 @@ -355,10 +351,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -368,7 +363,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -417,10 +412,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 @@ -479,7 +473,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) @@ -562,10 +556,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -574,7 +567,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -597,10 +590,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -610,7 +602,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -638,7 +630,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 4 Execution mode: vectorized, llap @@ -686,10 +678,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 5 @@ -817,10 +808,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -829,7 +819,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 4 @@ -859,10 +849,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -872,7 +861,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 6 @@ -902,10 +891,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -915,7 +903,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -968,10 +956,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 @@ -1030,7 +1017,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 7 @@ -1058,7 +1045,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 @@ -1154,10 +1141,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1166,7 +1152,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 5 @@ -1189,10 +1175,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1202,7 +1187,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 7 @@ -1225,10 +1210,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1238,7 +1222,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1266,7 +1250,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: llap @@ -1350,7 +1334,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 8 @@ -1378,7 +1362,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 @@ -1474,10 +1458,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1486,7 +1469,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 5 @@ -1509,10 +1492,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1522,7 +1504,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 7 @@ -1545,10 +1527,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1558,7 +1539,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1586,7 +1567,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: llap @@ -1670,7 +1651,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 8 @@ -1698,7 +1679,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 @@ -1794,10 +1775,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1806,7 +1786,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 5 @@ -1829,10 +1809,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1842,7 +1821,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 7 @@ -1865,10 +1844,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -1878,7 +1856,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -1906,7 +1884,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: llap @@ -1990,7 +1968,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 8 @@ -2018,7 +1996,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 @@ -2114,10 +2092,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -2126,7 +2103,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 5 @@ -2149,10 +2126,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -2162,7 +2138,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 7 @@ -2185,10 +2161,9 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized, llap @@ -2198,7 +2173,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -2226,7 +2201,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: llap @@ -2310,7 +2285,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 8 @@ -2338,7 +2313,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 diff --git ql/src/test/results/clientpositive/llap/vector_left_outer_join2.q.out ql/src/test/results/clientpositive/llap/vector_left_outer_join2.q.out index 591e283..c9b794e 100644 --- ql/src/test/results/clientpositive/llap/vector_left_outer_join2.q.out +++ ql/src/test/results/clientpositive/llap/vector_left_outer_join2.q.out @@ -371,7 +371,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(2)) Execution mode: vectorized, llap @@ -511,7 +511,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(2)) Execution mode: vectorized, llap @@ -650,7 +650,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(2)) Execution mode: vectorized, llap @@ -789,7 +789,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(2)) Execution mode: vectorized, llap 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 b8788fb..2e980b9 100644 --- ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out @@ -3364,10 +3364,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3395,7 +3394,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3476,10 +3475,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3507,7 +3505,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3590,10 +3588,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3621,7 +3618,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3699,10 +3696,9 @@ STAGE PLANS: className: VectorSelectOperator native: true Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3730,7 +3726,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3816,10 +3812,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3847,7 +3842,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3929,7 +3924,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -3956,10 +3951,9 @@ STAGE PLANS: className: VectorSelectOperator native: true Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4041,7 +4035,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4068,10 +4062,9 @@ STAGE PLANS: className: VectorSelectOperator native: true Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4150,7 +4143,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4174,10 +4167,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4257,10 +4249,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4288,7 +4279,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4382,10 +4373,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4413,7 +4403,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4495,10 +4485,9 @@ STAGE PLANS: className: VectorSelectOperator native: true Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4519,7 +4508,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4547,7 +4536,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4638,10 +4627,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4669,7 +4657,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4758,10 +4746,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false, One MapJoin Condition IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4789,7 +4776,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4817,7 +4804,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4905,10 +4892,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false, One MapJoin Condition IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4926,7 +4912,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -4951,7 +4937,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5048,7 +5034,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5066,7 +5052,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5091,7 +5077,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5192,7 +5178,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5217,7 +5203,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5235,7 +5221,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5336,7 +5322,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5361,7 +5347,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5379,7 +5365,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5482,7 +5468,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5507,7 +5493,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5525,7 +5511,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5650,10 +5636,9 @@ STAGE PLANS: nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true nativeConditionsNotMet: hive.vectorized.execution.mapjoin.native.enabled IS false Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5681,7 +5666,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5699,7 +5684,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5842,7 +5827,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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 Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -5924,10 +5909,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -5985,7 +5969,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6114,10 +6098,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6175,7 +6158,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6306,10 +6289,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6367,7 +6349,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6498,10 +6480,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6559,7 +6540,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6693,10 +6674,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6754,7 +6734,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6887,7 +6867,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -6946,10 +6926,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7082,7 +7061,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 93 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7141,10 +7120,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7274,7 +7252,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7325,10 +7303,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7456,10 +7433,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7517,7 +7493,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7659,10 +7635,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7721,7 +7696,7 @@ STAGE PLANS: className: VectorReduceSinkLongOperator 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -7860,10 +7835,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -7904,7 +7878,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -7963,7 +7937,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8102,10 +8076,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8163,7 +8136,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8303,10 +8276,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8364,7 +8336,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8422,7 +8394,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8557,10 +8529,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8592,7 +8563,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8643,7 +8614,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8775,7 +8746,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8807,7 +8778,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -8858,7 +8829,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9010,7 +8981,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9061,7 +9032,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9093,7 +9064,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9245,7 +9216,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9296,7 +9267,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9328,7 +9299,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9482,7 +9453,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9533,7 +9504,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9565,7 +9536,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9764,10 +9735,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 26 Data size: 2475 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9825,7 +9795,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -9857,7 +9827,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10086,7 +10056,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10180,10 +10150,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10192,7 +10161,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -10241,7 +10210,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10373,10 +10342,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10385,7 +10353,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -10434,7 +10402,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10568,10 +10536,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10580,7 +10547,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -10629,7 +10596,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10763,10 +10730,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10775,7 +10741,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -10824,7 +10790,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10961,10 +10927,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -10973,7 +10938,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -11022,7 +10987,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11155,7 +11120,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11217,10 +11182,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11229,7 +11193,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -11353,7 +11317,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 93 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11415,10 +11379,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11427,7 +11390,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -11548,7 +11511,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11602,10 +11565,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11614,7 +11576,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -11736,10 +11698,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11748,7 +11709,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -11797,7 +11758,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11942,10 +11903,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -11954,7 +11914,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -12004,7 +11964,7 @@ STAGE PLANS: className: VectorReduceSinkLongOperator 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12143,10 +12103,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -12187,7 +12146,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -12246,7 +12205,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12388,10 +12347,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12400,7 +12358,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -12449,7 +12407,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12589,10 +12547,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12650,7 +12607,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12708,7 +12665,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12843,10 +12800,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12878,7 +12834,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -12929,7 +12885,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13061,7 +13017,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13093,7 +13049,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13144,7 +13100,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13296,7 +13252,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13347,7 +13303,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13379,7 +13335,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13531,7 +13487,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13582,7 +13538,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13614,7 +13570,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13768,7 +13724,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13819,7 +13775,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -13851,7 +13807,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14056,10 +14012,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 26 Data size: 2475 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14068,7 +14023,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -14117,7 +14072,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14149,7 +14104,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14381,7 +14336,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14475,10 +14430,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14487,7 +14441,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -14536,7 +14490,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14668,10 +14622,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14680,7 +14633,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -14729,7 +14682,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14863,10 +14816,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -14875,7 +14827,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -14924,7 +14876,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15058,10 +15010,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15070,7 +15021,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -15119,7 +15070,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15256,10 +15207,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15268,7 +15218,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -15317,7 +15267,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15450,7 +15400,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 7 Data size: 651 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15512,10 +15462,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15524,7 +15473,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -15648,7 +15597,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 1 Data size: 93 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15710,10 +15659,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15722,7 +15670,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -15843,7 +15791,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 3 Data size: 279 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15897,10 +15845,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -15909,7 +15856,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -16031,10 +15978,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16043,7 +15989,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -16092,7 +16038,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16237,10 +16183,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 12 Data size: 1125 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16249,7 +16194,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -16299,7 +16244,7 @@ STAGE PLANS: className: VectorReduceSinkLongOperator 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16438,10 +16383,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int), _col3 (type: string) Execution mode: vectorized, llap @@ -16482,7 +16426,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -16541,7 +16485,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16683,10 +16627,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 24 Data size: 2250 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16695,7 +16638,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -16744,7 +16687,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16884,10 +16827,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -16945,7 +16887,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17003,7 +16945,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17138,10 +17080,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 48 Data size: 4501 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17173,7 +17114,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17224,7 +17165,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17356,7 +17297,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17388,7 +17329,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17439,7 +17380,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17591,7 +17532,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17642,7 +17583,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17674,7 +17615,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17826,7 +17767,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17877,7 +17818,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -17909,7 +17850,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18063,7 +18004,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 22 Data size: 2046 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18114,7 +18055,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18146,7 +18087,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18351,10 +18292,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 26 Data size: 2475 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18363,7 +18303,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true rowBatchContext: @@ -18412,7 +18352,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18444,7 +18384,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -18676,7 +18616,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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_string_partition.q.out ql/src/test/results/clientpositive/llap/vector_non_string_partition.q.out index 81dafa1..d835f5c 100644 --- ql/src/test/results/clientpositive/llap/vector_non_string_partition.q.out +++ ql/src/test/results/clientpositive/llap/vector_non_string_partition.q.out @@ -76,8 +76,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1024 Data size: 4096 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: tinyint) @@ -201,8 +201,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1024 Data size: 121205 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_nullsafe_join.q.out ql/src/test/results/clientpositive/llap/vector_nullsafe_join.q.out index 0d94b35..6552d2b 100644 --- ql/src/test/results/clientpositive/llap/vector_nullsafe_join.q.out +++ ql/src/test/results/clientpositive/llap/vector_nullsafe_join.q.out @@ -113,7 +113,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Execution mode: vectorized, llap @@ -255,7 +255,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Execution mode: vectorized, llap @@ -290,7 +290,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: int) Execution mode: vectorized, llap @@ -410,7 +410,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Execution mode: vectorized, llap @@ -438,7 +438,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: int) Execution mode: vectorized, llap @@ -599,7 +599,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -633,7 +633,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -752,7 +752,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -779,7 +779,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -996,7 +996,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Execution mode: vectorized, llap @@ -1138,7 +1138,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Execution mode: vectorized, llap @@ -1173,7 +1173,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: int) Execution mode: vectorized, llap @@ -1293,7 +1293,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Execution mode: vectorized, llap @@ -1321,7 +1321,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: int) Execution mode: vectorized, llap @@ -1482,7 +1482,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1516,7 +1516,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1635,7 +1635,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1662,7 +1662,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vector_number_compare_projection.q.out ql/src/test/results/clientpositive/llap/vector_number_compare_projection.q.out index dadafa2..5c20455 100644 --- ql/src/test/results/clientpositive/llap/vector_number_compare_projection.q.out +++ ql/src/test/results/clientpositive/llap/vector_number_compare_projection.q.out @@ -142,10 +142,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int) sort order: +++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2001 Data size: 273608 Basic stats: COMPLETE Column stats: NONE value expressions: _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean) Execution mode: vectorized, llap @@ -155,7 +154,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -267,10 +266,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint) sort order: ++++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2001 Data size: 273608 Basic stats: COMPLETE Column stats: NONE value expressions: _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean) Execution mode: vectorized, llap @@ -280,7 +278,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_orderby_5.q.out ql/src/test/results/clientpositive/llap/vector_orderby_5.q.out index 7d66676..04e6146 100644 --- ql/src/test/results/clientpositive/llap/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/llap/vector_orderby_5.q.out @@ -160,7 +160,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -200,10 +200,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: - Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 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 6c6c477..5c3f0e0 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join0.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join0.q.out @@ -165,7 +165,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 6 Data size: 550 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized, llap @@ -251,7 +251,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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 e687c5b..a39f59d 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join1.q.out @@ -329,7 +329,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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 @@ -497,7 +497,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 15 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -741,10 +741,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap @@ -785,7 +784,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 15 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -825,7 +824,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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 1dd24b0..e3104d2 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out @@ -327,10 +327,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap @@ -371,7 +370,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 20 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -411,7 +410,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 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_partition_diff_num_cols.q.out ql/src/test/results/clientpositive/llap/vector_partition_diff_num_cols.q.out index 09d8306..3e4ae94 100644 --- ql/src/test/results/clientpositive/llap/vector_partition_diff_num_cols.q.out +++ ql/src/test/results/clientpositive/llap/vector_partition_diff_num_cols.q.out @@ -124,10 +124,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -294,10 +293,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -464,10 +462,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -621,10 +618,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -778,10 +774,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out index 575daa1..77abb3b 100644 --- ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out +++ ql/src/test/results/clientpositive/llap/vector_partitioned_date_time.q.out @@ -289,8 +289,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: timestamp), _col4 (type: float) @@ -334,8 +334,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 25 Data size: 7200 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: timestamp), _col4 (type: float) @@ -471,7 +471,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 137 Data size: 39456 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -1199,8 +1199,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 137 Data size: 39448 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: timestamp), _col3 (type: float) @@ -1244,8 +1244,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 25 Data size: 1400 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: timestamp), _col3 (type: float) @@ -1405,7 +1405,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap @@ -2157,8 +2157,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 137 Data size: 39448 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: date), _col3 (type: float) @@ -2202,8 +2202,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 25 Data size: 1000 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: date), _col3 (type: float) @@ -2363,7 +2363,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 12 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_reduce1.q.out ql/src/test/results/clientpositive/llap/vector_reduce1.q.out index 4c17f95..794e595 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce1.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce1.q.out @@ -143,10 +143,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -155,7 +154,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_reduce2.q.out ql/src/test/results/clientpositive/llap/vector_reduce2.q.out index 4a467c2..170ea9c 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce2.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce2.q.out @@ -143,10 +143,9 @@ STAGE PLANS: key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string) sort order: +++ Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -155,7 +154,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_reduce3.q.out ql/src/test/results/clientpositive/llap/vector_reduce3.q.out index e9ac225..3d2992f 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce3.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce3.q.out @@ -143,10 +143,9 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -155,7 +154,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out index 47871e2..964e63d 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out @@ -80,7 +80,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 6102 Data size: 1440072 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -124,8 +124,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3051 Data size: 720036 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col4 (type: decimal(20,10)) diff --git ql/src/test/results/clientpositive/llap/vector_string_concat.q.out ql/src/test/results/clientpositive/llap/vector_string_concat.q.out index d08de2f..dbb41c0 100644 --- ql/src/test/results/clientpositive/llap/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/llap/vector_string_concat.q.out @@ -371,7 +371,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -412,8 +412,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out index aa63cb1..31b834a 100644 --- ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out +++ ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out @@ -262,10 +262,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out index 1fc3df0..c96b9df 100644 --- ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out +++ ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out @@ -179,7 +179,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -304,7 +304,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -431,7 +431,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 diff --git ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out index d3dd149..b219869 100644 --- ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out +++ ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out @@ -337,8 +337,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 10 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: int) diff --git ql/src/test/results/clientpositive/llap/vector_when_case_null.q.out ql/src/test/results/clientpositive/llap/vector_when_case_null.q.out index 49c5d9b..8104f3e 100644 --- ql/src/test/results/clientpositive/llap/vector_when_case_null.q.out +++ ql/src/test/results/clientpositive/llap/vector_when_case_null.q.out @@ -73,7 +73,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 5 Data size: 452 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorization_0.q.out ql/src/test/results/clientpositive/llap/vectorization_0.q.out index b9b0691..d919035 100644 --- ql/src/test/results/clientpositive/llap/vectorization_0.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_0.q.out @@ -61,10 +61,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized, llap @@ -102,10 +101,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 @@ -219,10 +217,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -260,10 +257,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reducer 3 Execution mode: vectorized, llap @@ -522,10 +518,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized, llap @@ -563,10 +558,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 @@ -680,10 +674,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -721,10 +714,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reducer 3 Execution mode: vectorized, llap @@ -983,10 +975,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized, llap @@ -1024,10 +1015,9 @@ STAGE PLANS: key expressions: _col0 (type: float) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 @@ -1141,10 +1131,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: double) Execution mode: vectorized, llap @@ -1182,10 +1171,9 @@ STAGE PLANS: key expressions: _col0 (type: double) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reducer 3 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorization_17.q.out ql/src/test/results/clientpositive/llap/vectorization_17.q.out index 9ea8483..735c015 100644 --- ql/src/test/results/clientpositive/llap/vectorization_17.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_17.q.out @@ -86,7 +86,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vectorization_7.q.out ql/src/test/results/clientpositive/llap/vectorization_7.q.out index 83e1e1d..9925959 100644 --- ql/src/test/results/clientpositive/llap/vectorization_7.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_7.q.out @@ -98,8 +98,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 7281 Data size: 1231410 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -334,8 +334,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 7281 Data size: 1231410 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorization_8.q.out ql/src/test/results/clientpositive/llap/vectorization_8.q.out index 4fd395f..42f12e9 100644 --- ql/src/test/results/clientpositive/llap/vectorization_8.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_8.q.out @@ -94,8 +94,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3060 Data size: 557456 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -317,8 +317,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3060 Data size: 557456 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorization_div0.q.out ql/src/test/results/clientpositive/llap/vectorization_div0.q.out index 37d05c8..2b5e5a8 100644 --- ql/src/test/results/clientpositive/llap/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_div0.q.out @@ -229,8 +229,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: decimal(22,21)) @@ -449,8 +449,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: double), _col4 (type: double), _col5 (type: double) diff --git ql/src/test/results/clientpositive/llap/vectorization_limit.q.out ql/src/test/results/clientpositive/llap/vectorization_limit.q.out index 5700005..c38a215 100644 --- ql/src/test/results/clientpositive/llap/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_limit.q.out @@ -122,8 +122,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 9173 Data size: 109584 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 value expressions: _col2 (type: smallint) @@ -391,7 +391,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 95 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 @@ -531,10 +531,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: tinyint) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 55052 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -725,7 +724,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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) Execution mode: vectorized, llap @@ -767,8 +766,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out index 8be0cac..f1fe221 100644 --- ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out @@ -118,8 +118,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 9173 Data size: 109584 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: smallint) diff --git ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out index 1e202f1..8268e11 100644 --- ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out @@ -1024,8 +1024,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 9898 Data size: 5632662 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -1323,8 +1323,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 8195 Data size: 3349694 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -1571,8 +1571,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 10922 Data size: 3594034 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: boolean) @@ -1877,8 +1877,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3868 Data size: 748844 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: timestamp) @@ -3284,10 +3284,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -3399,10 +3398,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -3586,10 +3584,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -3701,10 +3698,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -3816,10 +3812,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -3931,10 +3926,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -4046,10 +4040,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -4161,10 +4154,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorized_bucketmapjoin1.q.out ql/src/test/results/clientpositive/llap/vectorized_bucketmapjoin1.q.out index 277ca32..9d2d8a6 100644 --- ql/src/test/results/clientpositive/llap/vectorized_bucketmapjoin1.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_bucketmapjoin1.q.out @@ -138,7 +138,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -173,7 +173,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -273,7 +273,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap @@ -393,7 +393,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorized_case.q.out ql/src/test/results/clientpositive/llap/vectorized_case.q.out index b4703be..940b36e 100644 --- ql/src/test/results/clientpositive/llap/vectorized_case.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_case.q.out @@ -299,10 +299,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap @@ -427,10 +426,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out index d2d8136..c98ea9c 100644 --- ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out @@ -1270,10 +1270,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized, llap @@ -1311,10 +1310,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out index 3eb655d..0182a46 100644 --- ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out @@ -2525,7 +2525,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: true vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out index 5f4735f..e62d913 100644 --- ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out @@ -78,7 +78,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 90000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -121,7 +121,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 57 Data size: 10146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -145,10 +145,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap @@ -239,10 +238,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) @@ -317,7 +315,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 90000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -360,7 +358,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 57 Data size: 10146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) @@ -384,10 +382,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 552 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap @@ -478,10 +475,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 552 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) @@ -556,7 +552,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 90000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -599,7 +595,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 57 Data size: 10146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) @@ -623,10 +619,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 552 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap @@ -717,10 +712,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 552 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) @@ -796,7 +790,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 90000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -839,7 +833,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 57 Data size: 10146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -863,10 +857,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap @@ -910,7 +903,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 57 Data size: 10146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -934,10 +927,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap @@ -1030,10 +1022,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 7 @@ -1060,10 +1051,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) @@ -1139,7 +1129,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 90000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1182,7 +1172,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 57 Data size: 10146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) @@ -1206,10 +1196,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 552 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Select Operator @@ -1234,10 +1223,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap @@ -1328,10 +1316,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 552 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 6 @@ -1358,10 +1345,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) @@ -1436,7 +1422,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 500 Data size: 90000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1479,7 +1465,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 29 Data size: 5162 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -1503,10 +1489,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap @@ -1597,10 +1582,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) diff --git ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out index fa89c56..1c72876 100644 --- ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out @@ -125,7 +125,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out index fbe0b6c..37eb47e 100644 --- ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out @@ -101,10 +101,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized, llap @@ -147,7 +146,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 45 Data size: 181 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out index 18aab49..894c529 100644 --- ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out @@ -163,7 +163,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -791,7 +791,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1004,7 +1004,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1281,7 +1281,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1561,7 +1561,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1843,7 +1843,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -2234,7 +2234,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3028,7 +3028,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3304,7 +3304,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3654,7 +3654,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3934,7 +3934,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -4327,7 +4327,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -4914,7 +4914,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: diff --git ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out index e20ac03..7f04eba 100644 --- ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out @@ -54,7 +54,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -96,7 +96,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs diff --git ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out index eb2e45e..ed509de 100644 --- ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out @@ -126,10 +126,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized, llap @@ -139,7 +138,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -310,10 +309,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized, llap @@ -323,7 +321,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -494,10 +492,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean) Execution mode: vectorized, llap @@ -507,7 +504,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -678,10 +675,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized, llap @@ -691,7 +687,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -822,10 +818,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 1 Data size: 96 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/spark/vector_between_in.q.out ql/src/test/results/clientpositive/spark/vector_between_in.q.out index f1407e1..cd9dfc7 100644 --- ql/src/test/results/clientpositive/spark/vector_between_in.q.out +++ ql/src/test/results/clientpositive/spark/vector_between_in.q.out @@ -58,10 +58,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -69,7 +68,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -161,10 +160,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -260,10 +258,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(20,10)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -271,7 +268,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -363,10 +360,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -462,10 +458,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1365 Data size: 274112 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -473,7 +468,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -557,10 +552,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 10923 Data size: 2193503 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -568,7 +562,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -652,10 +646,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(20,10)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1365 Data size: 274112 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -663,7 +656,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -755,10 +748,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -1108,10 +1100,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: boolean) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1150,10 +1141,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1246,10 +1236,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: boolean) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1288,10 +1277,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1384,10 +1372,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: boolean) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1426,10 +1413,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1522,10 +1508,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: boolean) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1564,10 +1549,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out index 601f926..4454445 100644 --- ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out +++ ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out @@ -1279,10 +1279,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -1328,10 +1327,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/spark/vector_data_types.q.out ql/src/test/results/clientpositive/spark/vector_data_types.q.out index 3b088e3..a34173a 100644 --- ql/src/test/results/clientpositive/spark/vector_data_types.q.out +++ ql/src/test/results/clientpositive/spark/vector_data_types.q.out @@ -232,8 +232,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary) diff --git ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out index 9f1ea25..9ee1d95 100644 --- ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out @@ -83,10 +83,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: bigint), _col6 (type: decimal(23,14)), _col7 (type: decimal(23,14)), _col8 (type: decimal(33,14)), _col9 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out index bb81864..b7b0603 100644 --- ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out +++ ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out @@ -154,10 +154,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out index b05b2ca..55d1f3e 100644 --- ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out +++ ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out @@ -156,10 +156,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: tinyint), _col1 (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out index 8ee9a2e..440e92c 100644 --- ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out @@ -157,10 +157,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: boolean) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -199,10 +198,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: - Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/spark/vector_outer_join1.q.out ql/src/test/results/clientpositive/spark/vector_outer_join1.q.out index 169b0f8..1ab0a39 100644 --- ql/src/test/results/clientpositive/spark/vector_outer_join1.q.out +++ ql/src/test/results/clientpositive/spark/vector_outer_join1.q.out @@ -826,10 +826,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vector_outer_join2.q.out ql/src/test/results/clientpositive/spark/vector_outer_join2.q.out index d56c4ab..5a1a23d 100644 --- ql/src/test/results/clientpositive/spark/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/spark/vector_outer_join2.q.out @@ -407,10 +407,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vector_string_concat.q.out ql/src/test/results/clientpositive/spark/vector_string_concat.q.out index c76ce99..bea44e2 100644 --- ql/src/test/results/clientpositive/spark/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/spark/vector_string_concat.q.out @@ -369,8 +369,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -409,8 +409,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Reducer 3 diff --git ql/src/test/results/clientpositive/spark/vectorization_0.q.out ql/src/test/results/clientpositive/spark/vectorization_0.q.out index 714c21f..d8801f4 100644 --- ql/src/test/results/clientpositive/spark/vectorization_0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_0.q.out @@ -60,10 +60,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized @@ -100,10 +99,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 @@ -216,10 +214,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -256,10 +253,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: vectorized @@ -514,10 +510,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 32 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized @@ -554,10 +549,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 32 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 @@ -670,10 +664,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -710,10 +703,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: vectorized @@ -968,10 +960,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized @@ -1008,10 +999,9 @@ STAGE PLANS: key expressions: _col0 (type: float) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint) Reducer 3 @@ -1124,10 +1114,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: double) Execution mode: vectorized @@ -1164,10 +1153,9 @@ STAGE PLANS: key expressions: _col0 (type: double) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 3 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vectorization_17.q.out ql/src/test/results/clientpositive/spark/vectorization_17.q.out index a8f401b..32d1c0b 100644 --- ql/src/test/results/clientpositive/spark/vectorization_17.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_17.q.out @@ -84,7 +84,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/spark/vectorization_div0.q.out ql/src/test/results/clientpositive/spark/vectorization_div0.q.out index 923ff5e..81979d6 100644 --- ql/src/test/results/clientpositive/spark/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_div0.q.out @@ -227,8 +227,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: decimal(22,21)) @@ -445,8 +445,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: double), _col4 (type: double), _col5 (type: double) diff --git ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out index 98c0505..1eca688 100644 --- ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out @@ -1011,8 +1011,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 9898 Data size: 303864 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -1308,8 +1308,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -1554,8 +1554,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 10922 Data size: 335301 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: boolean) @@ -1858,8 +1858,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3868 Data size: 118746 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: timestamp) @@ -3251,10 +3251,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -3364,10 +3363,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -3549,10 +3547,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -3662,10 +3659,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -3775,10 +3771,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -3888,10 +3883,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -4001,10 +3995,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -4114,10 +4107,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vectorized_case.q.out ql/src/test/results/clientpositive/spark/vectorized_case.q.out index 30e17c4..ead71ad 100644 --- ql/src/test/results/clientpositive/spark/vectorized_case.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_case.q.out @@ -296,10 +296,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized @@ -422,10 +421,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out index b1b820e..64c3609 100644 --- ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out @@ -161,7 +161,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -433,7 +433,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -511,7 +511,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -783,7 +783,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -993,7 +993,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1266,7 +1266,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1542,7 +1542,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1820,7 +1820,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -1898,7 +1898,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -2134,7 +2134,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -2209,7 +2209,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -2993,7 +2993,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3265,7 +3265,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3610,7 +3610,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3886,7 +3886,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -3964,7 +3964,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -4277,7 +4277,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -4857,7 +4857,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -5448,7 +5448,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -5824,7 +5824,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -6199,7 +6199,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -6547,7 +6547,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -6941,7 +6941,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: @@ -7301,7 +7301,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Path -> Alias: diff --git ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out index 9aec2aa..adf6e65 100644 --- ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out @@ -51,10 +51,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -62,7 +61,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 5 @@ -93,10 +92,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -104,7 +102,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 diff --git ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out index 3fb3925..3892165 100644 --- ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out @@ -125,10 +125,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized @@ -137,7 +136,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -307,10 +306,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized @@ -319,7 +317,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -489,10 +487,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean) Execution mode: vectorized @@ -501,7 +498,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -671,10 +668,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized @@ -683,7 +679,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -813,10 +809,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark 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: 1 Data size: 96 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out index 779c6ef..fa5ef8f 100644 --- ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out +++ ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out @@ -173,10 +173,6 @@ STAGE PLANS: Reduce Operator Tree: Group By Operator aggregations: min(VALUE._col0), max(VALUE._col1), sum(VALUE._col2), avg(VALUE._col3) - Group By Vectorization: - vectorOutput: false - native: false - projectedOutputColumns: null mode: mergepartial outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 624 Basic stats: COMPLETE Column stats: NONE diff --git ql/src/test/results/clientpositive/tez/vector_auto_smb_mapjoin_14.q.out ql/src/test/results/clientpositive/tez/vector_auto_smb_mapjoin_14.q.out index 1269ed8..bc15fcd 100644 --- ql/src/test/results/clientpositive/tez/vector_auto_smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/tez/vector_auto_smb_mapjoin_14.q.out @@ -283,10 +283,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 @@ -528,7 +527,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 5 Data size: 465 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -582,7 +581,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 5 Data size: 465 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) @@ -1284,7 +1283,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -1326,7 +1325,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/tez/vector_between_columns.q.out ql/src/test/results/clientpositive/tez/vector_between_columns.q.out index 0b0af7d..fb88de5 100644 --- ql/src/test/results/clientpositive/tez/vector_between_columns.q.out +++ ql/src/test/results/clientpositive/tez/vector_between_columns.q.out @@ -161,10 +161,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 5 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: smallint) Execution mode: vectorized @@ -173,7 +172,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true @@ -327,10 +326,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 5 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: smallint) Execution mode: vectorized @@ -339,7 +337,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true diff --git ql/src/test/results/clientpositive/tez/vector_between_in.q.out ql/src/test/results/clientpositive/tez/vector_between_in.q.out index 9011e6c..509471b 100644 --- ql/src/test/results/clientpositive/tez/vector_between_in.q.out +++ ql/src/test/results/clientpositive/tez/vector_between_in.q.out @@ -59,10 +59,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -70,7 +69,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -163,10 +162,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -263,10 +261,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(20,10)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -274,7 +271,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -367,10 +364,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -467,10 +463,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1365 Data size: 274112 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -478,7 +473,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -563,10 +558,9 @@ STAGE PLANS: key expressions: _col0 (type: date) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 10923 Data size: 2193503 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -574,7 +568,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -659,10 +653,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(20,10)) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1365 Data size: 274112 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -670,7 +663,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 2 @@ -763,10 +756,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -1119,7 +1111,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1158,10 +1150,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1257,7 +1248,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1296,10 +1287,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1395,7 +1385,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1434,10 +1424,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 @@ -1533,7 +1522,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -1572,10 +1561,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 diff --git ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out index dc93d1a..40a0dc4 100644 --- ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out +++ ql/src/test/results/clientpositive/tez/vector_binary_join_groupby.q.out @@ -181,10 +181,9 @@ STAGE PLANS: Reduce Output Operator sort order: Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -226,7 +225,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)) Execution mode: vectorized @@ -349,7 +348,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized @@ -566,7 +565,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkLongOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: binary) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/tez/vector_cast_constant.q.out ql/src/test/results/clientpositive/tez/vector_cast_constant.q.out index ba43096..b386c11 100644 --- ql/src/test/results/clientpositive/tez/vector_cast_constant.q.out +++ ql/src/test/results/clientpositive/tez/vector_cast_constant.q.out @@ -181,10 +181,6 @@ STAGE PLANS: Reduce Operator Tree: Group By Operator aggregations: avg(VALUE._col0), avg(VALUE._col1), avg(VALUE._col2) - Group By Vectorization: - vectorOutput: false - native: false - projectedOutputColumns: null keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1, _col2, _col3 diff --git ql/src/test/results/clientpositive/tez/vector_char_2.q.out ql/src/test/results/clientpositive/tez/vector_char_2.q.out index 0fbaaca..9ed165c 100644 --- ql/src/test/results/clientpositive/tez/vector_char_2.q.out +++ ql/src/test/results/clientpositive/tez/vector_char_2.q.out @@ -113,7 +113,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -156,8 +156,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 250 Data size: 49500 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -310,7 +310,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 @@ -353,8 +353,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 250 Data size: 49500 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) diff --git ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out index f38bf30..355a676 100644 --- ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out +++ ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out @@ -187,10 +187,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: char(10)) Execution mode: vectorized @@ -199,7 +198,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -232,7 +231,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Execution mode: vectorized @@ -343,7 +342,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Execution mode: vectorized @@ -397,10 +396,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: char(20)) Execution mode: vectorized @@ -409,7 +407,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reducer 3 @@ -526,10 +524,9 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: string) Execution mode: vectorized @@ -538,7 +535,7 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Map 3 @@ -571,7 +568,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3 Data size: 273 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/tez/vector_char_simple.q.out ql/src/test/results/clientpositive/tez/vector_char_simple.q.out index e2b7814..a1015bb 100644 --- ql/src/test/results/clientpositive/tez/vector_char_simple.q.out +++ ql/src/test/results/clientpositive/tez/vector_char_simple.q.out @@ -251,8 +251,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true diff --git ql/src/test/results/clientpositive/tez/vector_coalesce.q.out ql/src/test/results/clientpositive/tez/vector_coalesce.q.out index f0d5ef4..40df703 100644 --- ql/src/test/results/clientpositive/tez/vector_coalesce.q.out +++ ql/src/test/results/clientpositive/tez/vector_coalesce.q.out @@ -41,8 +41,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -146,8 +146,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -250,8 +250,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -355,8 +355,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -458,8 +458,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true diff --git ql/src/test/results/clientpositive/tez/vector_coalesce_2.q.out ql/src/test/results/clientpositive/tez/vector_coalesce_2.q.out index d38f8ae..c7eb029 100644 --- ql/src/test/results/clientpositive/tez/vector_coalesce_2.q.out +++ ql/src/test/results/clientpositive/tez/vector_coalesce_2.q.out @@ -236,7 +236,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkStringOperator native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, Not ACID UPDATE or DELETE IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 4 Data size: 510 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/tez/vector_non_string_partition.q.out ql/src/test/results/clientpositive/tez/vector_non_string_partition.q.out index 436a3c3..647a095 100644 --- ql/src/test/results/clientpositive/tez/vector_non_string_partition.q.out +++ ql/src/test/results/clientpositive/tez/vector_non_string_partition.q.out @@ -76,8 +76,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1024 Data size: 4096 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: tinyint) @@ -200,8 +200,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1024 Data size: 121205 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/tez/vectorization_div0.q.out ql/src/test/results/clientpositive/tez/vectorization_div0.q.out index 46a6c14..cb7dc90 100644 --- ql/src/test/results/clientpositive/tez/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/tez/vectorization_div0.q.out @@ -228,8 +228,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1365 Data size: 174720 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: decimal(22,21)) @@ -447,8 +447,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 1365 Data size: 65520 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: double), _col4 (type: double), _col5 (type: double) diff --git ql/src/test/results/clientpositive/tez/vectorization_limit.q.out ql/src/test/results/clientpositive/tez/vectorization_limit.q.out index 35b617e..ad6a159 100644 --- ql/src/test/results/clientpositive/tez/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/tez/vectorization_limit.q.out @@ -121,8 +121,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 9173 Data size: 109584 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 value expressions: _col2 (type: smallint) @@ -387,7 +387,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 95 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 @@ -526,10 +526,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: tinyint) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: Uniform Hash IS false + className: VectorReduceSinkObjectHashOperator + 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: 6144 Data size: 55052 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized Map Vectorization: @@ -719,7 +718,7 @@ STAGE PLANS: 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 buckets IS true, No TopN IS true, Uniform Hash IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS 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) Execution mode: vectorized @@ -760,8 +759,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: No TopN IS false Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 Reducer 3 diff --git ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out index 99d578f..357c760 100644 --- ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out +++ ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out @@ -198,8 +198,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized @@ -308,8 +308,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 100 Data size: 29638 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_char_2.q.out ql/src/test/results/clientpositive/vector_char_2.q.out index adfd5f0..60bd50f 100644 --- ql/src/test/results/clientpositive/vector_char_2.q.out +++ ql/src/test/results/clientpositive/vector_char_2.q.out @@ -107,8 +107,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -285,8 +285,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 500 Data size: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) diff --git ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out index dafbc17..d1bdadb 100644 --- ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out +++ ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out @@ -204,8 +204,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: char(10)) Execution mode: vectorized @@ -335,8 +335,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: char(20)) Execution mode: vectorized @@ -468,8 +468,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 3 Data size: 323 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: string) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_char_simple.q.out ql/src/test/results/clientpositive/vector_char_simple.q.out index 3dbddb8..c7b00fd 100644 --- ql/src/test/results/clientpositive/vector_char_simple.q.out +++ ql/src/test/results/clientpositive/vector_char_simple.q.out @@ -228,8 +228,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true diff --git ql/src/test/results/clientpositive/vector_coalesce.q.out ql/src/test/results/clientpositive/vector_coalesce.q.out index 6f74ec9..f158236 100644 --- ql/src/test/results/clientpositive/vector_coalesce.q.out +++ ql/src/test/results/clientpositive/vector_coalesce.q.out @@ -37,8 +37,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -122,8 +122,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -206,8 +206,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -291,8 +291,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true @@ -375,8 +375,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Execution mode: vectorized Map Vectorization: enabled: true diff --git ql/src/test/results/clientpositive/vector_coalesce_2.q.out ql/src/test/results/clientpositive/vector_coalesce_2.q.out index 1518235..6778499 100644 --- ql/src/test/results/clientpositive/vector_coalesce_2.q.out +++ ql/src/test/results/clientpositive/vector_coalesce_2.q.out @@ -220,8 +220,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 4 Data size: 510 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_count.q.out ql/src/test/results/clientpositive/vector_count.q.out index 23d5099..3473759 100644 --- ql/src/test/results/clientpositive/vector_count.q.out +++ ql/src/test/results/clientpositive/vector_count.q.out @@ -93,8 +93,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE value expressions: _col5 (type: bigint) Execution mode: vectorized @@ -196,8 +196,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE value expressions: _col4 (type: bigint), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col9 (type: bigint) Execution mode: vectorized @@ -283,8 +283,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE value expressions: d (type: int) Execution mode: vectorized @@ -373,8 +373,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 7 Data size: 100 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_data_types.q.out ql/src/test/results/clientpositive/vector_data_types.q.out index beb8602..09641ca 100644 --- ql/src/test/results/clientpositive/vector_data_types.q.out +++ ql/src/test/results/clientpositive/vector_data_types.q.out @@ -221,8 +221,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary) diff --git ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out index 2940048..0146186 100644 --- ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out @@ -80,8 +80,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: bigint), _col6 (type: decimal(23,14)), _col7 (type: decimal(23,14)), _col8 (type: decimal(33,14)), _col9 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_decimal_expressions.q.out ql/src/test/results/clientpositive/vector_decimal_expressions.q.out index 819555c..627acfd 100644 --- ql/src/test/results/clientpositive/vector_decimal_expressions.q.out +++ ql/src/test/results/clientpositive/vector_decimal_expressions.q.out @@ -59,8 +59,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 455 Data size: 78809 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_decimal_round.q.out ql/src/test/results/clientpositive/vector_decimal_round.q.out index b1fd628..62461eb 100644 --- ql/src/test/results/clientpositive/vector_decimal_round.q.out +++ ql/src/test/results/clientpositive/vector_decimal_round.q.out @@ -383,8 +383,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(11,0)) Execution mode: vectorized @@ -466,8 +466,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_decimal_round_2.q.out ql/src/test/results/clientpositive/vector_decimal_round_2.q.out index 6223268..7c0a3fb 100644 --- ql/src/test/results/clientpositive/vector_decimal_round_2.q.out +++ ql/src/test/results/clientpositive/vector_decimal_round_2.q.out @@ -71,8 +71,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(21,0)), _col2 (type: decimal(22,1)), _col3 (type: decimal(23,2)), _col4 (type: decimal(24,3)), _col5 (type: decimal(21,0)), _col6 (type: decimal(21,0)), _col7 (type: decimal(21,0)), _col8 (type: decimal(21,0)), _col9 (type: decimal(21,0)), _col10 (type: decimal(21,0)), _col11 (type: decimal(21,0)), _col12 (type: decimal(21,0)) Execution mode: vectorized @@ -204,8 +204,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(21,0)), _col2 (type: decimal(22,1)), _col3 (type: decimal(23,2)), _col4 (type: decimal(24,3)), _col5 (type: decimal(25,4)), _col6 (type: decimal(21,0)), _col7 (type: decimal(21,0)), _col8 (type: decimal(21,0)), _col9 (type: decimal(21,0)), _col10 (type: decimal(21,0)), _col11 (type: decimal(21,0)), _col12 (type: decimal(22,1)), _col13 (type: decimal(23,2)), _col14 (type: decimal(24,3)), _col15 (type: decimal(25,4)), _col16 (type: decimal(21,0)), _col17 (type: decimal(21,0)), _col18 (type: decimal(21,0)), _col19 (type: decimal(21,0)) Execution mode: vectorized @@ -364,8 +364,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(21,0)), _col2 (type: decimal(21,0)), _col3 (type: decimal(21,0)), _col4 (type: decimal(21,0)), _col5 (type: decimal(21,0)), _col6 (type: decimal(21,0)), _col7 (type: decimal(21,0)), _col8 (type: decimal(21,0)), _col9 (type: decimal(21,0)), _col10 (type: decimal(21,0)), _col11 (type: decimal(21,0)), _col12 (type: decimal(21,0)), _col13 (type: decimal(21,0)), _col14 (type: decimal(21,0)), _col15 (type: decimal(21,0)), _col16 (type: decimal(21,0)), _col17 (type: decimal(22,1)), _col18 (type: decimal(23,2)), _col19 (type: decimal(24,3)), _col20 (type: decimal(25,4)), _col21 (type: decimal(26,5)), _col22 (type: decimal(27,6)), _col23 (type: decimal(28,7)), _col24 (type: decimal(29,8)), _col25 (type: decimal(30,9)), _col26 (type: decimal(31,10)), _col27 (type: decimal(32,11)), _col28 (type: decimal(33,12)), _col29 (type: decimal(34,13)), _col31 (type: decimal(35,14)), _col32 (type: decimal(36,15)), _col33 (type: decimal(37,16)) Execution mode: vectorized @@ -513,8 +513,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(30,9)) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_distinct_2.q.out ql/src/test/results/clientpositive/vector_distinct_2.q.out index bcd6ba1..4bf3f0f 100644 --- ql/src/test/results/clientpositive/vector_distinct_2.q.out +++ ql/src/test/results/clientpositive/vector_distinct_2.q.out @@ -151,8 +151,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_empty_where.q.out ql/src/test/results/clientpositive/vector_empty_where.q.out index 7b5c456..a1fb19d 100644 --- ql/src/test/results/clientpositive/vector_empty_where.q.out +++ ql/src/test/results/clientpositive/vector_empty_where.q.out @@ -56,8 +56,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -155,8 +155,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -262,8 +262,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -369,8 +369,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No DISTINCT columns IS false Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_groupby4.q.out ql/src/test/results/clientpositive/vector_groupby4.q.out index 0423e47..d22e9c1 100644 --- ql/src/test/results/clientpositive/vector_groupby4.q.out +++ ql/src/test/results/clientpositive/vector_groupby4.q.out @@ -62,8 +62,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_groupby6.q.out ql/src/test/results/clientpositive/vector_groupby6.q.out index a801942..e323197 100644 --- ql/src/test/results/clientpositive/vector_groupby6.q.out +++ ql/src/test/results/clientpositive/vector_groupby6.q.out @@ -62,8 +62,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 500 Data size: 88000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_groupby_3.q.out ql/src/test/results/clientpositive/vector_groupby_3.q.out index 2af0705..2d0ca34 100644 --- ql/src/test/results/clientpositive/vector_groupby_3.q.out +++ ql/src/test/results/clientpositive/vector_groupby_3.q.out @@ -153,8 +153,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_groupby_reduce.q.out ql/src/test/results/clientpositive/vector_groupby_reduce.q.out index a464409..56d3e46 100644 --- ql/src/test/results/clientpositive/vector_groupby_reduce.q.out +++ ql/src/test/results/clientpositive/vector_groupby_reduce.q.out @@ -271,8 +271,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -422,8 +422,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -683,8 +683,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int) Execution mode: vectorized @@ -947,8 +947,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1000 Data size: 88276 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: int) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_grouping_sets.q.out ql/src/test/results/clientpositive/vector_grouping_sets.q.out index 5c9909e..a4199ed 100644 --- ql/src/test/results/clientpositive/vector_grouping_sets.q.out +++ ql/src/test/results/clientpositive/vector_grouping_sets.q.out @@ -177,8 +177,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -291,8 +291,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_if_expr.q.out ql/src/test/results/clientpositive/vector_if_expr.q.out index 4f30acf..043940b 100644 --- ql/src/test/results/clientpositive/vector_if_expr.q.out +++ ql/src/test/results/clientpositive/vector_if_expr.q.out @@ -44,8 +44,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_include_no_sel.q.out ql/src/test/results/clientpositive/vector_include_no_sel.q.out index a357c21..e5e3a52 100644 --- ql/src/test/results/clientpositive/vector_include_no_sel.q.out +++ ql/src/test/results/clientpositive/vector_include_no_sel.q.out @@ -250,8 +250,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.vectorized.execution.reducesink.new.enabled IS false, hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.vectorized.execution.reducesink.new.enabled IS false, hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_interval_1.q.out ql/src/test/results/clientpositive/vector_interval_1.q.out index b7a8e5a..f53a2c2 100644 --- ql/src/test/results/clientpositive/vector_interval_1.q.out +++ ql/src/test/results/clientpositive/vector_interval_1.q.out @@ -85,8 +85,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: interval_year_month), _col4 (type: interval_day_time) Execution mode: vectorized @@ -194,8 +194,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: interval_year_month), _col3 (type: interval_year_month), _col5 (type: interval_year_month), _col6 (type: interval_year_month) Execution mode: vectorized @@ -311,8 +311,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col5 (type: interval_day_time), _col6 (type: interval_day_time) Execution mode: vectorized @@ -440,8 +440,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: date), _col2 (type: date), _col3 (type: date), _col4 (type: date), _col5 (type: date), _col6 (type: date), _col7 (type: timestamp), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp) Execution mode: vectorized @@ -581,8 +581,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp), _col7 (type: timestamp), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: timestamp), _col11 (type: timestamp), _col12 (type: timestamp) Execution mode: vectorized @@ -704,8 +704,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time) Execution mode: vectorized @@ -809,8 +809,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time) Execution mode: vectorized @@ -920,8 +920,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col4 (type: interval_day_time), _col5 (type: interval_day_time), _col6 (type: interval_day_time) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out index 4908fca..75250e3 100644 --- ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out +++ ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out @@ -93,8 +93,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: date), _col2 (type: date), _col3 (type: date), _col4 (type: date), _col5 (type: date), _col6 (type: date) Execution mode: vectorized @@ -258,8 +258,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time) Execution mode: vectorized @@ -423,8 +423,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp) Execution mode: vectorized @@ -583,8 +583,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -701,8 +701,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp) Execution mode: vectorized @@ -868,8 +868,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col4 (type: interval_day_time) Execution mode: vectorized @@ -1035,8 +1035,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_non_string_partition.q.out ql/src/test/results/clientpositive/vector_non_string_partition.q.out index d41fb55..018a193 100644 --- ql/src/test/results/clientpositive/vector_non_string_partition.q.out +++ ql/src/test/results/clientpositive/vector_non_string_partition.q.out @@ -70,8 +70,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 1024 Data size: 113013 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: tinyint) @@ -173,8 +173,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 1024 Data size: 113013 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_orderby_5.q.out ql/src/test/results/clientpositive/vector_orderby_5.q.out index 5c5e2a0..e156e08 100644 --- ql/src/test/results/clientpositive/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/vector_orderby_5.q.out @@ -154,8 +154,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_outer_join1.q.out ql/src/test/results/clientpositive/vector_outer_join1.q.out index 565ddac..87ff0c4 100644 --- ql/src/test/results/clientpositive/vector_outer_join1.q.out +++ ql/src/test/results/clientpositive/vector_outer_join1.q.out @@ -699,8 +699,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_outer_join2.q.out ql/src/test/results/clientpositive/vector_outer_join2.q.out index 4f63113..f8e757e 100644 --- ql/src/test/results/clientpositive/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/vector_outer_join2.q.out @@ -346,8 +346,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_outer_join3.q.out ql/src/test/results/clientpositive/vector_outer_join3.q.out index 53377db..870d419 100644 --- ql/src/test/results/clientpositive/vector_outer_join3.q.out +++ ql/src/test/results/clientpositive/vector_outer_join3.q.out @@ -242,7 +242,7 @@ left outer join small_alltypesorc_a hd on hd.cstring1 = c.cstring1 ) t1 POSTHOOK: type: QUERY -{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 6]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col1"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2, 6]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} +{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 6]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col1"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2, 6]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} PREHOOK: query: select count(*) from (select c.cstring1 from small_alltypesorc_a c left outer join small_alltypesorc_a cd @@ -282,7 +282,7 @@ left outer join small_alltypesorc_a hd on hd.cstring1 = c.cstring1 ) t1 POSTHOOK: type: QUERY -{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cstring2 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[6, 7]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} +{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cstring2 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[6, 7]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} PREHOOK: query: select count(*) from (select c.cstring1 from small_alltypesorc_a c left outer join small_alltypesorc_a cd @@ -322,7 +322,7 @@ left outer join small_alltypesorc_a hd on hd.cstring1 = c.cstring1 and hd.cint = c.cint ) t1 POSTHOOK: type: QUERY -{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cbigint (type: bigint), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cint (type: int), cbigint (type: bigint), cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 3, 6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2, 3, 6, 7]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} +{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cbigint (type: bigint), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cint (type: int), cbigint (type: bigint), cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 3, 6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2, 3, 6, 7]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} PREHOOK: query: select count(*) from (select c.cstring1 from small_alltypesorc_a c left outer join small_alltypesorc_a cd diff --git ql/src/test/results/clientpositive/vector_outer_join4.q.out ql/src/test/results/clientpositive/vector_outer_join4.q.out index 324b67e..3953cf8 100644 --- ql/src/test/results/clientpositive/vector_outer_join4.q.out +++ ql/src/test/results/clientpositive/vector_outer_join4.q.out @@ -780,7 +780,7 @@ left outer join small_alltypesorc_b hd on hd.ctinyint = c.ctinyint ) t1 POSTHOOK: type: QUERY -{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), cint (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 2]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 36 Data size: 8082 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0, 2]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} +{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), cint (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 2]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 36 Data size: 8082 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0, 2]","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"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}} PREHOOK: query: select count(*) from (select c.ctinyint from small_alltypesorc_b c left outer join small_alltypesorc_b cd diff --git ql/src/test/results/clientpositive/vector_reduce1.q.out ql/src/test/results/clientpositive/vector_reduce1.q.out index 1f4273a..2a985b1 100644 --- ql/src/test/results/clientpositive/vector_reduce1.q.out +++ ql/src/test/results/clientpositive/vector_reduce1.q.out @@ -139,8 +139,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_reduce2.q.out ql/src/test/results/clientpositive/vector_reduce2.q.out index 8855c39..27ea4ff 100644 --- ql/src/test/results/clientpositive/vector_reduce2.q.out +++ ql/src/test/results/clientpositive/vector_reduce2.q.out @@ -139,8 +139,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_reduce3.q.out ql/src/test/results/clientpositive/vector_reduce3.q.out index b432b2e..2264366 100644 --- ql/src/test/results/clientpositive/vector_reduce3.q.out +++ ql/src/test/results/clientpositive/vector_reduce3.q.out @@ -139,8 +139,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: diff --git ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out index ee2f05d..0f0548b 100644 --- ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out +++ ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out @@ -74,8 +74,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 6102 Data size: 1440072 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col4 (type: decimal(20,10)) diff --git ql/src/test/results/clientpositive/vector_string_concat.q.out ql/src/test/results/clientpositive/vector_string_concat.q.out index ca7d0c8..a121785 100644 --- ql/src/test/results/clientpositive/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/vector_string_concat.q.out @@ -361,8 +361,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_varchar_simple.q.out ql/src/test/results/clientpositive/vector_varchar_simple.q.out index d28d5ea..1bed203 100644 --- ql/src/test/results/clientpositive/vector_varchar_simple.q.out +++ ql/src/test/results/clientpositive/vector_varchar_simple.q.out @@ -306,8 +306,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 10 Data size: 2150 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: int) diff --git ql/src/test/results/clientpositive/vector_when_case_null.q.out ql/src/test/results/clientpositive/vector_when_case_null.q.out index cf57736..7476bbb 100644 --- ql/src/test/results/clientpositive/vector_when_case_null.q.out +++ ql/src/test/results/clientpositive/vector_when_case_null.q.out @@ -67,8 +67,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 5 Data size: 452 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorization_7.q.out ql/src/test/results/clientpositive/vectorization_7.q.out index 9568810..e534296 100644 --- ql/src/test/results/clientpositive/vectorization_7.q.out +++ ql/src/test/results/clientpositive/vectorization_7.q.out @@ -92,8 +92,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 7281 Data size: 1565441 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -306,8 +306,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 7281 Data size: 1565441 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorization_8.q.out ql/src/test/results/clientpositive/vectorization_8.q.out index fc456f0..7650c5f 100644 --- ql/src/test/results/clientpositive/vectorization_8.q.out +++ ql/src/test/results/clientpositive/vectorization_8.q.out @@ -88,8 +88,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -289,8 +289,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorization_div0.q.out ql/src/test/results/clientpositive/vectorization_div0.q.out index a30f7c0..83dff79 100644 --- ql/src/test/results/clientpositive/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/vectorization_div0.q.out @@ -219,8 +219,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: decimal(22,21)) @@ -417,8 +417,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: double), _col4 (type: double), _col5 (type: double) diff --git ql/src/test/results/clientpositive/vectorization_limit.q.out ql/src/test/results/clientpositive/vectorization_limit.q.out index 17cdd51..3ea3564 100644 --- ql/src/test/results/clientpositive/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/vectorization_limit.q.out @@ -112,8 +112,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.3 value expressions: _col2 (type: smallint) @@ -353,8 +353,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.3 Execution mode: vectorized @@ -477,8 +477,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false, No DISTINCT columns IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, No DISTINCT columns IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.3 Execution mode: vectorized @@ -638,8 +638,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorization_offset_limit.q.out ql/src/test/results/clientpositive/vectorization_offset_limit.q.out index b398e42..5da2ad1 100644 --- ql/src/test/results/clientpositive/vectorization_offset_limit.q.out +++ ql/src/test/results/clientpositive/vectorization_offset_limit.q.out @@ -108,8 +108,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, No TopN IS false Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: smallint) diff --git ql/src/test/results/clientpositive/vectorized_case.q.out ql/src/test/results/clientpositive/vectorized_case.q.out index 2596788..2be619a 100644 --- ql/src/test/results/clientpositive/vectorized_case.q.out +++ ql/src/test/results/clientpositive/vectorized_case.q.out @@ -287,8 +287,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized @@ -398,8 +398,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorized_date_funcs.q.out ql/src/test/results/clientpositive/vectorized_date_funcs.q.out index 24b1ad4..6fe6735 100644 --- ql/src/test/results/clientpositive/vectorized_date_funcs.q.out +++ ql/src/test/results/clientpositive/vectorized_date_funcs.q.out @@ -1250,8 +1250,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out index 4d4e4ae..5e0f699 100644 --- ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out +++ ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out @@ -119,8 +119,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorized_timestamp.q.out ql/src/test/results/clientpositive/vectorized_timestamp.q.out index 4945e4a..df7ee50 100644 --- ql/src/test/results/clientpositive/vectorized_timestamp.q.out +++ ql/src/test/results/clientpositive/vectorized_timestamp.q.out @@ -308,8 +308,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out index 527affd..aaee6e7 100644 --- ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out @@ -122,8 +122,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized @@ -287,8 +287,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized @@ -452,8 +452,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean) Execution mode: vectorized @@ -617,8 +617,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized @@ -742,8 +742,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No buckets IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false, Uniform Hash IS false + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: bigint), _col3 (type: bigint) Execution mode: vectorized