diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java index fc5aea5..a4dbe0b 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.vector.reducesink; import java.io.IOException; +import java.io.Serializable; import java.util.Arrays; import java.util.Properties; @@ -29,6 +30,7 @@ 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.TopNHash; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow; import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; @@ -57,11 +59,13 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hive.common.util.HashCodeUtil; +import com.google.common.base.Preconditions; + /** * This class is common operator class for native vectorized reduce sink. */ public abstract class VectorReduceSinkCommonOperator extends TerminalOperator - implements VectorizationContextRegion { + implements Serializable, TopNHash.BinaryCollector, VectorizationContextRegion { private static final long serialVersionUID = 1L; private static final String CLASS_NAME = VectorReduceSinkCommonOperator.class.getName(); @@ -122,6 +126,9 @@ protected transient HiveKey keyWritable; protected transient BytesWritable valueBytesWritable; + // Picks topN K:V pairs from input. + protected transient TopNHash reducerHash; + // Where to write our key and value pairs. private transient OutputCollector out; @@ -329,10 +336,46 @@ protected void initializeOp(Configuration hconf) throws HiveException { valueBytesWritable = new BytesWritable(); + int limit = conf.getTopN(); + float memUsage = conf.getTopNMemoryUsage(); + + if (limit >= 0 && memUsage > 0) { + reducerHash = new TopNHash(); + reducerHash.initialize(limit, memUsage, conf.isMapGroupBy(), this, conf, hconf); + } + batchCounter = 0; } - protected void collect(BytesWritable keyWritable, Writable valueWritable) throws IOException { + // The collect method override for TopNHash.BinaryCollector + @Override + public void collect(byte[] key, byte[] value, int hash) throws IOException { + HiveKey keyWritable = new HiveKey(key, hash); + BytesWritable valueWritable = new BytesWritable(value); + doCollect(keyWritable, valueWritable); + } + + protected void collect(HiveKey keyWritable, BytesWritable valueWritable) + throws HiveException, IOException { + if (reducerHash != null) { + // NOTE: partColsIsNull is only used for PTF, which isn't supported yet. + final int firstIndex = + reducerHash.tryStoreKey(keyWritable, /* partColsIsNull */ false); + + if (firstIndex == TopNHash.EXCLUDE) return; // Nothing to do. + + if (firstIndex == TopNHash.FORWARD) { + doCollect(keyWritable, valueWritable); + } else { + Preconditions.checkState(firstIndex >= 0); + reducerHash.storeValue(firstIndex, keyWritable.hashCode(), valueWritable, false); + } + } else { + doCollect(keyWritable, valueWritable); + } + } + + private void doCollect(HiveKey keyWritable, BytesWritable valueWritable) throws IOException { // Since this is a terminal operator, update counters explicitly - // forward is not called if (null != out) { @@ -360,8 +403,12 @@ protected void collect(BytesWritable keyWritable, Writable valueWritable) throws @Override protected void closeOp(boolean abort) throws HiveException { + if (!abort && reducerHash != null) { + reducerHash.flush(); + } super.closeOp(abort); out = null; + reducerHash = null; if (isLogInfoEnabled) { LOG.info(toString() + ": records written - " + numRows); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java index 6312c44..bd7d6cb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java @@ -177,7 +177,6 @@ protected void initializeOp(Configuration hconf) throws HiveException { 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]; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java index 0bb0f22..3b3624d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java @@ -20,6 +20,8 @@ import java.sql.Date; import java.sql.Timestamp; +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.ql.exec.MapredContext; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; @@ -39,6 +41,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableBinaryObjectInspector; +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; +import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Text; @@ -336,6 +340,25 @@ private void setOutputCol(ColumnVector colVec, int i, Object value) { BytesWritable bw = (BytesWritable) value; BytesColumnVector bv = (BytesColumnVector) colVec; bv.setVal(i, bw.getBytes(), 0, bw.getLength()); + } else if (outputOI instanceof WritableHiveIntervalYearMonthObjectInspector) { + LongColumnVector lv = (LongColumnVector) colVec; + HiveIntervalYearMonth iym; + if (value instanceof HiveIntervalYearMonth) { + iym = (HiveIntervalYearMonth) value; + } else { + iym = ((WritableHiveIntervalYearMonthObjectInspector) outputOI).getPrimitiveJavaObject(value); + } + long l = iym.getTotalMonths(); + lv.vector[i] = l; + } else if (outputOI instanceof WritableHiveIntervalDayTimeObjectInspector) { + IntervalDayTimeColumnVector idtv = (IntervalDayTimeColumnVector) colVec; + HiveIntervalDayTime idt; + if (value instanceof HiveIntervalDayTime) { + idt = (HiveIntervalDayTime) value; + } else { + idt = ((WritableHiveIntervalDayTimeObjectInspector) outputOI).getPrimitiveJavaObject(value); + } + idtv.set(i, idt); } else { throw new RuntimeException("Unhandled object type " + outputOI.getTypeName() + " inspector class " + outputOI.getClass().getName() + diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index f0df2e9..2025c24 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 @@ -3025,7 +3025,10 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, String engine = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); - boolean hasTopN = (desc.getTopN() >= 0); + int limit = desc.getTopN(); + float memUsage = desc.getTopNMemoryUsage(); + + boolean hasPTFTopN = (limit >= 0 && memUsage > 0 && desc.isPTFReduceSink()); boolean hasDistinctColumns = (desc.getDistinctColumnIndices().size() > 0); @@ -3202,7 +3205,7 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, // Remember the condition variables for EXPLAIN regardless. vectorDesc.setIsVectorizationReduceSinkNativeEnabled(isVectorizationReduceSinkNativeEnabled); vectorDesc.setEngine(engine); - vectorDesc.setHasTopN(hasTopN); + vectorDesc.setHasPTFTopN(hasPTFTopN); vectorDesc.setHasDistinctColumns(hasDistinctColumns); vectorDesc.setIsKeyBinarySortable(isKeyBinarySortable); vectorDesc.setIsValueLazyBinary(isValueLazyBinary); @@ -3215,7 +3218,7 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, if (!isVectorizationReduceSinkNativeEnabled || !isTezOrSpark || (useUniformHash && (hasEmptyBuckets || hasNoPartitions)) || - hasTopN || + hasPTFTopN || hasDistinctColumns || !isKeyBinarySortable || !isValueLazyBinary || diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java index 79d19b5..c059db6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java @@ -544,8 +544,8 @@ public ReduceSinkOperatorExplainVectorization(ReduceSinkDesc reduceSinkDesc, Vec engineInSupported, engineInSupportedCondName), new VectorizationCondition( - !vectorReduceSinkDesc.getHasTopN(), - "No TopN"), + !vectorReduceSinkDesc.getHasPTFTopN(), + "No PTF TopN"), new VectorizationCondition( !vectorReduceSinkDesc.getHasDistinctColumns(), "No DISTINCT columns"), diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java index d6230af..91d5be7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java @@ -64,7 +64,7 @@ public VectorReduceSinkInfo getVectorReduceSinkInfo() { private boolean isVectorizationReduceSinkNativeEnabled; private String engine; - private boolean hasTopN; + private boolean hasPTFTopN; private boolean hasDistinctColumns; private boolean isKeyBinarySortable; private boolean isValueLazyBinary; @@ -85,11 +85,11 @@ public void setEngine(String engine) { public String getEngine() { return engine; } - public void setHasTopN(boolean hasTopN) { - this.hasTopN = hasTopN; + public void setHasPTFTopN(boolean hasPTFTopN) { + this.hasPTFTopN = hasPTFTopN; } - public boolean getHasTopN() { - return hasTopN; + public boolean getHasPTFTopN() { + return hasPTFTopN; } public void setHasDistinctColumns(boolean hasDistinctColumns) { this.hasDistinctColumns = hasDistinctColumns; diff --git ql/src/test/queries/clientpositive/vectorization_limit.q ql/src/test/queries/clientpositive/vectorization_limit.q index a4c54f2..8044484 100644 --- ql/src/test/queries/clientpositive/vectorization_limit.q +++ ql/src/test/queries/clientpositive/vectorization_limit.q @@ -3,6 +3,8 @@ set hive.explain.user=false; SET hive.vectorized.execution.enabled=true; set hive.fetch.task.conversion=none; +-- SORT_QUERY_RESULTS + explain vectorization SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; @@ -11,31 +13,31 @@ set hive.limit.pushdown.memory.usage=0.3f; -- HIVE-3562 Some limit can be pushed down to map stage - c/p parts from limit_pushdown -explain vectorization expression +explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; -- deduped RS -explain vectorization expression +explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; -- distincts -explain vectorization expression +explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20; select distinct(ctinyint) from alltypesorc limit 20; -explain vectorization expression +explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20; select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20; -- limit zero -explain vectorization expression +explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; -- 2MR (applied to last RS) -explain vectorization expression +explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; diff --git ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out index 526662d..57ddc96 100644 --- ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out +++ ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out @@ -83,7 +83,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -126,7 +126,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs @@ -261,7 +261,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -304,7 +304,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs 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 5b17144..a2ce365 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 a0ac248..3b44bc3 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 @@ -287,7 +287,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -531,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -586,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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) @@ -1292,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1335,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 cb9674c..d6f6ec8 100644 --- ql/src/test/results/clientpositive/llap/vector_between_columns.q.out +++ ql/src/test/results/clientpositive/llap/vector_between_columns.q.out @@ -164,7 +164,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -331,7 +331,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 9b58d47..ae1b2d2 100644 --- ql/src/test/results/clientpositive/llap/vector_between_in.q.out +++ ql/src/test/results/clientpositive/llap/vector_between_in.q.out @@ -61,7 +61,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -165,7 +165,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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,7 +265,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -369,7 +369,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -469,7 +469,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -565,7 +565,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -661,7 +661,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -765,7 +765,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1119,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1161,7 +1161,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 +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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1299,7 +1299,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 +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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1437,7 +1437,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 +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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1575,7 +1575,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 339ec2c..a35659a 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 @@ -185,7 +185,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -228,7 +228,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -268,7 +268,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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, llap @@ -383,7 +383,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -433,7 +433,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 50 Data size: 14819 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 @@ -630,7 +630,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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_bucket.q.out ql/src/test/results/clientpositive/llap/vector_bucket.q.out index 7b57223..6dd0cfb 100644 --- ql/src/test/results/clientpositive/llap/vector_bucket.q.out +++ ql/src/test/results/clientpositive/llap/vector_bucket.q.out @@ -52,7 +52,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 26 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string) 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 c330097..65fafb0 100644 --- ql/src/test/results/clientpositive/llap/vector_char_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_char_2.q.out @@ -111,10 +111,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: char(20)) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkStringOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -155,10 +154,9 @@ STAGE PLANS: key expressions: _col0 (type: char(20)) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -309,10 +307,9 @@ STAGE PLANS: sort order: - Map-reduce partition columns: _col0 (type: char(20)) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkStringOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -353,10 +350,9 @@ STAGE PLANS: key expressions: _col0 (type: char(20)) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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 e0df3c0..ab67adc 100644 --- ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out +++ ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out @@ -188,7 +188,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -231,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -343,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -399,7 +399,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -527,7 +527,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -570,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 d3ab3f2..5e1dea8 100644 --- ql/src/test/results/clientpositive/llap/vector_char_simple.q.out +++ ql/src/test/results/clientpositive/llap/vector_char_simple.q.out @@ -77,7 +77,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,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 Reducer 2 @@ -251,10 +251,9 @@ STAGE PLANS: className: VectorLimitOperator 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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: @@ -262,7 +261,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_coalesce.q.out ql/src/test/results/clientpositive/llap/vector_coalesce.q.out index 578f849..358d8ae 100644 --- ql/src/test/results/clientpositive/llap/vector_coalesce.q.out +++ ql/src/test/results/clientpositive/llap/vector_coalesce.q.out @@ -39,10 +39,9 @@ STAGE PLANS: projectedOutputColumns: [6, 2, 4, 1, 16] selectExpressions: VectorCoalesce(columns [12, 6, 13, 14, 15])(children: ConstantVectorExpression(val null) -> 12:string, col 6, CastLongToString(col 2) -> 13:String, VectorUDFAdaptor(null(cfloat)) -> 14:string, CastLongToString(col 1) -> 15:String) -> 16:string Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -50,7 +49,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 @@ -145,10 +144,9 @@ STAGE PLANS: projectedOutputColumns: [5, 2, 15] selectExpressions: VectorCoalesce(columns [12, 14, 13])(children: ConstantVectorExpression(val null) -> 12:double, DoubleColAddDoubleColumn(col 5, col 13)(children: FuncLog2LongToDouble(col 2) -> 13:double) -> 14:double, ConstantVectorExpression(val 0.0) -> 13:double) -> 15: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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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: @@ -156,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 @@ -250,10 +248,9 @@ STAGE PLANS: native: true projectedOutputColumns: [] Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -261,7 +258,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 @@ -356,10 +353,9 @@ STAGE PLANS: projectedOutputColumns: [8, 9, 12] selectExpressions: VectorCoalesce(columns [8, 9])(children: col 8, col 9) -> 12:timestamp Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -367,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 @@ -460,10 +456,9 @@ STAGE PLANS: native: true projectedOutputColumns: [] Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -471,7 +466,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_coalesce_2.q.out ql/src/test/results/clientpositive/llap/vector_coalesce_2.q.out index 2a3eff5..d57d39f 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 a6e2f59..5fa5a82 100644 --- ql/src/test/results/clientpositive/llap/vector_count.q.out +++ ql/src/test/results/clientpositive/llap/vector_count.q.out @@ -99,7 +99,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 TopN IS true, BinarySortableSerDe for keys IS 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 PTF 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) @@ -208,7 +208,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 TopN IS true, BinarySortableSerDe for keys IS 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 PTF 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) @@ -301,7 +301,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 TopN IS true, BinarySortableSerDe for keys IS 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 PTF 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) @@ -397,7 +397,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 TopN IS true, BinarySortableSerDe for keys IS 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 PTF 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 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 3c03787..18e16cf 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1331,7 +1331,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 f67bc25..b9bb0a2 100644 --- ql/src/test/results/clientpositive/llap/vector_data_types.q.out +++ ql/src/test/results/clientpositive/llap/vector_data_types.q.out @@ -235,10 +235,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -249,7 +248,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_aggregate.q.out ql/src/test/results/clientpositive/llap/vector_decimal_aggregate.q.out index 44119b2..ab38382 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 02888e7..2847807 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out @@ -63,10 +63,9 @@ STAGE PLANS: key expressions: _col0 (type: decimal(25,14)), _col1 (type: decimal(26,14)), _col2 (type: decimal(38,13)), _col3 (type: decimal(38,17)), _col4 (type: decimal(12,10)), _col5 (type: int), _col6 (type: smallint), _col7 (type: tinyint), _col8 (type: bigint), _col9 (type: boolean), _col10 (type: double), _col11 (type: float), _col12 (type: string), _col13 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 455 Data size: 78809 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -76,7 +75,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_mapjoin.q.out ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out index 08aca71..c62e25a 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 8bd80cf..00bb50a 100644 --- ql/src/test/results/clientpositive/llap/vector_decimal_round.q.out +++ ql/src/test/results/clientpositive/llap/vector_decimal_round.q.out @@ -73,7 +73,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(11,0)) Execution mode: vectorized, llap @@ -175,7 +175,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized, llap @@ -501,7 +501,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -603,7 +603,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 6083a10..a3bf091 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 @@ -77,7 +77,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -228,7 +228,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -406,7 +406,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -573,7 +573,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 84022f8..f17583f 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 f582ca3..81dfac6 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -111,7 +111,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -217,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -266,7 +266,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -380,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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,7 +429,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -543,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -592,7 +592,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 3b3b801..ffeab2c 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby4.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby4.q.out @@ -69,7 +69,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -109,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 7e21493..5bfa9b5 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby6.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby6.q.out @@ -69,7 +69,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -109,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 5063f06..0242cbd 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 cfc87fb..1c67983 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -266,7 +266,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 1b1ec9e..7bfbd6f 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby_mapjoin.q.out @@ -97,7 +97,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string) Execution mode: vectorized, llap @@ -142,7 +142,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -181,7 +181,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 205 Data size: 17835 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs @@ -247,7 +247,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -287,7 +287,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 0ba6195..39e81f3 100644 --- ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out +++ ql/src/test/results/clientpositive/llap/vector_groupby_reduce.q.out @@ -278,10 +278,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false + className: VectorReduceSinkLongOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -319,10 +318,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 500 Data size: 44138 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Reducer 3 @@ -481,7 +479,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -541,7 +539,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -763,7 +761,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -826,7 +824,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1049,7 +1047,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1112,7 +1110,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 9d51982..92a6a6c 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 f06086e..32d1001 100644 --- ql/src/test/results/clientpositive/llap/vector_if_expr.q.out +++ ql/src/test/results/clientpositive/llap/vector_if_expr.q.out @@ -50,7 +50,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 029b3f5..efd49cd 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,7 +203,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) @@ -268,7 +268,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/llap/vector_inner_join.q.out ql/src/test/results/clientpositive/llap/vector_inner_join.q.out index fc7b9a3..a1c5f81 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 debf5ab..8d4f12e 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_1.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_1.q.out @@ -91,7 +91,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -219,7 +219,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -355,7 +355,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -503,7 +503,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -662,7 +662,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -803,7 +803,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -926,7 +926,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1055,7 +1055,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 7037f97..1509926 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_2.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_2.q.out @@ -145,7 +145,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -352,7 +352,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -559,7 +559,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -766,7 +766,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -965,7 +965,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1160,7 +1160,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1345,7 +1345,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1530,7 +1530,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1725,7 +1725,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1920,7 +1920,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 aadb6e7..1d14092 100644 --- ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out +++ ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out @@ -99,7 +99,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -282,7 +282,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -465,7 +465,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -641,10 +641,10 @@ STAGE PLANS: key expressions: CAST( 5-5 AS INTERVAL YEAR TO MONTH) (type: interval_year_month) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false + className: VectorReduceSinkObjectHashOperator + keyExpressions: VectorUDFAdaptor(CAST( 5-5 AS INTERVAL YEAR TO MONTH)) -> 2:interval_year_month + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -654,8 +654,8 @@ STAGE PLANS: enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - allNative: false - usesVectorUDFAdaptor: false + allNative: true + usesVectorUDFAdaptor: true vectorized: true Reducer 2 Execution mode: vectorized, llap @@ -784,7 +784,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -969,7 +969,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1154,7 +1154,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 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 6d828a5..0024fea 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 6af0959..394393e 100644 --- ql/src/test/results/clientpositive/llap/vector_join30.q.out +++ ql/src/test/results/clientpositive/llap/vector_join30.q.out @@ -73,7 +73,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -114,7 +114,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -175,7 +175,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -234,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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) @@ -319,7 +319,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -353,7 +353,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -414,7 +414,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -473,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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) @@ -558,7 +558,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -592,7 +592,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -630,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -680,7 +680,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -810,7 +810,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -851,7 +851,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -893,7 +893,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -958,7 +958,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1017,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1045,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1143,7 +1143,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1177,7 +1177,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1212,7 +1212,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1250,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1334,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1362,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1460,7 +1460,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1494,7 +1494,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1529,7 +1529,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1567,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1651,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1679,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1777,7 +1777,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1811,7 +1811,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1846,7 +1846,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1884,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1968,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1996,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -2094,7 +2094,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -2128,7 +2128,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -2163,7 +2163,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -2201,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -2285,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -2313,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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_join_part_col_char.q.out ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out index 95dcba9..8d40a6d 100644 --- ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out +++ ql/src/test/results/clientpositive/llap/vector_join_part_col_char.q.out @@ -111,24 +111,24 @@ Stage-0 Reducer 2 llap File Output Operator [FS_10] Merge Join Operator [MERGEJOIN_21] (rows=2 width=431) - Conds:RS_23._col2=RS_28._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + Conds:RS_6._col2=RS_7._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] <-Map 1 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_23] + SHUFFLE [RS_6] PartitionCols:_col2 - Select Operator [SEL_22] (rows=2 width=134) + Select Operator [SEL_2] (rows=2 width=134) Output:["_col0","_col1","_col2"] TableScan [TS_0] (rows=2 width=236) default@char_tbl1,c1,Tbl:COMPLETE,Col:PARTIAL,Output:["name","age"] - Dynamic Partitioning Event Operator [EVENT_26] (rows=1 width=134) - Group By Operator [GBY_25] (rows=1 width=134) + Dynamic Partitioning Event Operator [EVENT_20] (rows=1 width=134) + Group By Operator [GBY_19] (rows=1 width=134) Output:["_col0"],keys:_col0 - Select Operator [SEL_24] (rows=2 width=134) + Select Operator [SEL_18] (rows=2 width=134) Output:["_col0"] - Please refer to the previous Select Operator [SEL_22] + Please refer to the previous Select Operator [SEL_2] <-Map 3 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_28] + SHUFFLE [RS_7] PartitionCols:_col2 - Select Operator [SEL_27] (rows=2 width=89) + Select Operator [SEL_5] (rows=2 width=89) Output:["_col0","_col1","_col2"] TableScan [TS_3] (rows=2 width=190) default@char_tbl2,c2,Tbl:COMPLETE,Col:PARTIAL,Output:["name","age"] diff --git ql/src/test/results/clientpositive/llap/vector_left_outer_join2.q.out ql/src/test/results/clientpositive/llap/vector_left_outer_join2.q.out index c9b794e..20f6acc 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 6c6d0f3..445e585 100644 --- ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out +++ ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out @@ -3366,7 +3366,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -3394,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -3477,7 +3477,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -3505,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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,7 +3590,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -3618,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -3698,7 +3698,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -3726,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -3814,7 +3814,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -3842,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -3924,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -3953,7 +3953,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4035,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4064,7 +4064,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4143,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4169,7 +4169,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4251,7 +4251,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4279,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4375,7 +4375,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4403,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4487,7 +4487,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4508,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4536,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4629,7 +4629,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4657,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4748,7 +4748,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4776,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4804,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4894,7 +4894,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -4912,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -4937,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5034,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5052,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5077,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5178,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5203,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5221,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5322,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5347,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5365,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5468,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5493,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5511,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5638,7 +5638,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -5666,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5684,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5827,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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: @@ -5911,7 +5911,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -5969,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -6102,7 +6102,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -6160,7 +6160,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -6295,7 +6295,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -6353,7 +6353,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -6488,7 +6488,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -6546,7 +6546,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -6684,7 +6684,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -6742,7 +6742,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -6877,7 +6877,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -6938,7 +6938,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -7073,7 +7073,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -7134,7 +7134,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -7266,7 +7266,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -7319,7 +7319,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -7451,7 +7451,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -7509,7 +7509,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -7655,7 +7655,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -7714,7 +7714,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -7857,7 +7857,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -7898,7 +7898,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -7957,7 +7957,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8100,7 +8100,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -8158,7 +8158,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8302,7 +8302,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -8360,7 +8360,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8418,7 +8418,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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,7 +8557,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -8589,7 +8589,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8640,7 +8640,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8774,7 +8774,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8806,7 +8806,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -8857,7 +8857,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9011,7 +9011,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9062,7 +9062,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9094,7 +9094,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9248,7 +9248,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9299,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9331,7 +9331,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9487,7 +9487,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9538,7 +9538,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9570,7 +9570,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9773,7 +9773,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -9831,7 +9831,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -9863,7 +9863,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -10094,7 +10094,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -10190,7 +10190,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -10248,7 +10248,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -10384,7 +10384,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -10442,7 +10442,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -10580,7 +10580,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -10638,7 +10638,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -10776,7 +10776,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -10834,7 +10834,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -10975,7 +10975,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -11033,7 +11033,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -11168,7 +11168,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -11232,7 +11232,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -11367,7 +11367,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -11431,7 +11431,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -11563,7 +11563,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -11619,7 +11619,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -11754,7 +11754,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -11812,7 +11812,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -11961,7 +11961,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -12020,7 +12020,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12163,7 +12163,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -12204,7 +12204,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12263,7 +12263,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12409,7 +12409,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -12467,7 +12467,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12611,7 +12611,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -12669,7 +12669,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12727,7 +12727,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12866,7 +12866,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -12898,7 +12898,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -12949,7 +12949,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13083,7 +13083,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13115,7 +13115,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13166,7 +13166,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13320,7 +13320,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13371,7 +13371,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13403,7 +13403,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13557,7 +13557,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13608,7 +13608,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13640,7 +13640,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13796,7 +13796,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13847,7 +13847,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -13879,7 +13879,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -14088,7 +14088,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -14146,7 +14146,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -14178,7 +14178,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -14412,7 +14412,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -14508,7 +14508,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -14566,7 +14566,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -14702,7 +14702,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -14760,7 +14760,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -14898,7 +14898,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -14956,7 +14956,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -15094,7 +15094,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -15152,7 +15152,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -15293,7 +15293,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -15351,7 +15351,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -15486,7 +15486,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -15550,7 +15550,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -15685,7 +15685,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -15749,7 +15749,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -15881,7 +15881,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -15937,7 +15937,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -16072,7 +16072,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -16130,7 +16130,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -16279,7 +16279,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -16338,7 +16338,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -16481,7 +16481,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -16522,7 +16522,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -16581,7 +16581,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -16727,7 +16727,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -16785,7 +16785,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -16929,7 +16929,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -16987,7 +16987,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17045,7 +17045,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17184,7 +17184,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -17216,7 +17216,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17267,7 +17267,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17401,7 +17401,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17433,7 +17433,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17484,7 +17484,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17638,7 +17638,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17689,7 +17689,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17721,7 +17721,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17875,7 +17875,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17926,7 +17926,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -17958,7 +17958,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -18114,7 +18114,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -18165,7 +18165,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -18197,7 +18197,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -18406,7 +18406,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -18464,7 +18464,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -18496,7 +18496,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -18730,7 +18730,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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_mapjoin_reduce.q.out ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out index ef67ec2..d3586e0 100644 --- ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out @@ -59,7 +59,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs @@ -101,7 +101,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 17 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col2 (type: int) Execution mode: vectorized, llap @@ -155,7 +155,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs @@ -326,7 +326,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 50 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs @@ -369,7 +369,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 17 Data size: 272 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) Execution mode: vectorized, llap @@ -423,7 +423,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: no inputs diff --git ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out index 6edc474..10051d1 100644 --- ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out +++ ql/src/test/results/clientpositive/llap/vector_non_constant_in_expr.q.out @@ -21,7 +21,7 @@ STAGE PLANS: alias: alltypesorc Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (cint) IN (ctinyint, cbigint) (type: boolean) + predicate: (cint) IN (UDFToInteger(ctinyint), UDFToInteger(cbigint)) (type: boolean) Statistics: Num rows: 6144 Data size: 1546640 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean) @@ -40,7 +40,7 @@ STAGE PLANS: enabled: true enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - notVectorizedReason: Predicate expression for FILTER operator: Cannot vectorize IN() - casting a column is not supported. Column type is int but the common type is bigint + notVectorizedReason: Predicate expression for FILTER operator: Vectorizing IN expression only supported for constant values vectorized: false Stage: Stage-0 diff --git ql/src/test/results/clientpositive/llap/vector_non_string_partition.q.out ql/src/test/results/clientpositive/llap/vector_non_string_partition.q.out index d835f5c..de62ca2 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 @@ -74,10 +74,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1024 Data size: 4096 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: tinyint) @@ -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 @@ -199,10 +198,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1024 Data size: 121205 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -212,7 +210,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_nullsafe_join.q.out ql/src/test/results/clientpositive/llap/vector_nullsafe_join.q.out index 6552d2b..6225986 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 5c20455..aa1b9d8 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 @@ -144,7 +144,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -268,7 +268,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/llap/vector_order_null.q.out ql/src/test/results/clientpositive/llap/vector_order_null.q.out index 9df6f7a..eb61d02 100644 --- ql/src/test/results/clientpositive/llap/vector_order_null.q.out +++ ql/src/test/results/clientpositive/llap/vector_order_null.q.out @@ -105,7 +105,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -223,7 +223,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -341,7 +341,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -459,7 +459,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -577,7 +577,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -695,7 +695,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -813,7 +813,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -931,7 +931,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -1049,7 +1049,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -1167,7 +1167,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs @@ -1285,7 +1285,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: no inputs diff --git ql/src/test/results/clientpositive/llap/vector_orderby_5.q.out ql/src/test/results/clientpositive/llap/vector_orderby_5.q.out index 2a8eda6..7faf892 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -202,7 +202,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 5c3f0e0..48d4e5c 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 c089d00..19c4df4 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -743,7 +743,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -784,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -824,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 f1319a9..7b821a9 100644 --- ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/llap/vector_outer_join2.q.out @@ -329,7 +329,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -370,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 3e4ae94..ec79647 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 @@ -126,7 +126,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -295,7 +295,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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,7 +464,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -620,7 +620,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -776,7 +776,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 2870d9e..e511bd5 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 @@ -287,10 +287,9 @@ STAGE PLANS: key expressions: _col5 (type: int), _col2 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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 TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: timestamp), _col4 (type: float) @@ -301,7 +300,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,7 +309,7 @@ STAGE PLANS: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true groupByVectorOutput: true - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reduce Operator Tree: @@ -332,10 +331,9 @@ STAGE PLANS: key expressions: _col5 (type: int), _col2 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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 +469,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1197,10 +1195,9 @@ STAGE PLANS: key expressions: _col4 (type: int), _col5 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -1211,7 +1208,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 @@ -1220,7 +1217,7 @@ STAGE PLANS: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true groupByVectorOutput: true - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reduce Operator Tree: @@ -1242,10 +1239,9 @@ STAGE PLANS: key expressions: _col4 (type: int), _col5 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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 +1401,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -2155,10 +2151,9 @@ STAGE PLANS: key expressions: _col4 (type: int), _col5 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -2169,7 +2164,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 @@ -2178,7 +2173,7 @@ STAGE PLANS: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true groupByVectorOutput: true - allNative: false + allNative: true usesVectorUDFAdaptor: false vectorized: true Reduce Operator Tree: @@ -2200,10 +2195,9 @@ STAGE PLANS: key expressions: _col4 (type: int), _col5 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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 +2357,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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_ptf_part_simple.q.out ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out index 9929550..dbf90b0 100644 --- ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out +++ ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out @@ -143,7 +143,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: double) Execution mode: vectorized, llap @@ -368,7 +368,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -593,7 +593,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -812,7 +812,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: double) Execution mode: vectorized, llap @@ -1005,7 +1005,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -1198,7 +1198,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -1391,7 +1391,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -1626,7 +1626,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 12792 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: decimal(38,18)) Execution mode: vectorized, llap @@ -1819,7 +1819,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 12792 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: decimal(38,18)) Execution mode: vectorized, llap @@ -2032,7 +2032,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_bigint (type: bigint) Execution mode: vectorized, llap @@ -2225,7 +2225,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_bigint (type: bigint) Execution mode: vectorized, llap @@ -2412,7 +2412,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -2576,7 +2576,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap @@ -2741,7 +2741,7 @@ STAGE PLANS: className: VectorReduceSinkMultiKeyOperator keyExpressions: VectorUDFAdaptor(CASE WHEN ((p_mfgr = 'Manufacturer#2')) THEN (2000-01-01 00:00:00.0) ELSE (null) END)(children: StringGroupColEqualStringScalar(col 0, val Manufacturer#2) -> 3:boolean) -> 4:timestamp native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2907,7 +2907,7 @@ STAGE PLANS: className: VectorReduceSinkObjectHashOperator keyExpressions: VectorUDFAdaptor(CASE WHEN ((p_mfgr = 'Manufacturer#2')) THEN (2000-01-01 00:00:00.0) ELSE (null) END)(children: StringGroupColEqualStringScalar(col 0, val Manufacturer#2) -> 3:boolean) -> 4:timestamp native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 40 Data size: 9048 Basic stats: COMPLETE Column stats: NONE value expressions: p_retailprice (type: double) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vector_reduce1.q.out ql/src/test/results/clientpositive/llap/vector_reduce1.q.out index 794e595..4a4e710 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce1.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce1.q.out @@ -145,7 +145,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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_reduce2.q.out ql/src/test/results/clientpositive/llap/vector_reduce2.q.out index 170ea9c..d937146 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce2.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce2.q.out @@ -145,7 +145,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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_reduce3.q.out ql/src/test/results/clientpositive/llap/vector_reduce3.q.out index 3d2992f..382841f 100644 --- ql/src/test/results/clientpositive/llap/vector_reduce3.q.out +++ ql/src/test/results/clientpositive/llap/vector_reduce3.q.out @@ -145,7 +145,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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_reduce_groupby_decimal.q.out ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out index 964e63d..7348af8 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 @@ -78,10 +78,9 @@ STAGE PLANS: sort order: ++++ Map-reduce partition columns: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14)) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkMultiKeyOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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)) @@ -122,10 +121,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: decimal(20,10)), _col3 (type: decimal(23,14)) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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 8b19c58..fb9e121 100644 --- ql/src/test/results/clientpositive/llap/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/llap/vector_string_concat.q.out @@ -369,10 +369,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkOperator - native: 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 + className: VectorReduceSinkStringOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -410,10 +409,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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 TopN Hash Memory Usage: 0.1 Reducer 3 diff --git ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out index c036d69..6b4ca6c 100644 --- ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out +++ ql/src/test/results/clientpositive/llap/vector_string_decimal.q.out @@ -62,28 +62,44 @@ STAGE PLANS: TableScan alias: orc_decimal Statistics: Num rows: 4 Data size: 448 Basic stats: COMPLETE Column stats: NONE + TableScan Vectorization: + native: true + projectedOutputColumns: [0] Filter Operator - predicate: (id) IN ('100000000', '200000000') (type: boolean) + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: FilterDoubleColumnInList(col 1, values [1.0E8, 2.0E8])(children: CastDecimalToDouble(col 0) -> 1:double) -> boolean + predicate: (UDFToDouble(id)) IN (1.0E8, 2.0E8) (type: boolean) Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: id (type: decimal(18,0)) outputColumnNames: _col0 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumns: [0] Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: false Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: enabled: true enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true + groupByVectorOutput: true inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - notVectorizedReason: Predicate expression for FILTER operator: Cannot vectorize IN() - casting a column is not supported. Column type is decimal(18,0) but the common type is string - vectorized: false + allNative: false + usesVectorUDFAdaptor: false + vectorized: true Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out index 31b834a..3a431b6 100644 --- ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out +++ ql/src/test/results/clientpositive/llap/vector_tablesample_rows.q.out @@ -264,7 +264,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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_simple.q.out ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out index b219869..52f78da 100644 --- ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out +++ ql/src/test/results/clientpositive/llap/vector_varchar_simple.q.out @@ -93,7 +93,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 @@ -213,7 +213,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 @@ -335,10 +335,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 10 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: int) @@ -349,7 +348,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_when_case_null.q.out ql/src/test/results/clientpositive/llap/vector_when_case_null.q.out index 8104f3e..06dde80 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 d2897ba..73fb9b1 100644 --- ql/src/test/results/clientpositive/llap/vectorization_0.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_0.q.out @@ -63,7 +63,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -103,7 +103,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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,7 +219,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -259,7 +259,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -520,7 +520,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -560,7 +560,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -676,7 +676,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -716,7 +716,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -977,7 +977,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1017,7 +1017,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1133,7 +1133,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1173,7 +1173,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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_7.q.out ql/src/test/results/clientpositive/llap/vectorization_7.q.out index 9925959..ba49bed 100644 --- ql/src/test/results/clientpositive/llap/vectorization_7.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_7.q.out @@ -96,10 +96,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 7281 Data size: 1231410 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -109,7 +108,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 @@ -332,10 +331,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 7281 Data size: 1231410 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -345,7 +343,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_8.q.out ql/src/test/results/clientpositive/llap/vectorization_8.q.out index 42f12e9..9e9f2c7 100644 --- ql/src/test/results/clientpositive/llap/vectorization_8.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_8.q.out @@ -92,10 +92,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3060 Data size: 557456 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -105,7 +104,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 @@ -315,10 +314,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3060 Data size: 557456 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -328,7 +326,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_div0.q.out ql/src/test/results/clientpositive/llap/vectorization_div0.q.out index 2b5e5a8..2ff5c54 100644 --- ql/src/test/results/clientpositive/llap/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_div0.q.out @@ -227,10 +227,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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)) @@ -241,7 +240,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 @@ -447,10 +446,9 @@ STAGE PLANS: key expressions: _col0 (type: double), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -461,7 +459,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_limit.q.out ql/src/test/results/clientpositive/llap/vectorization_limit.q.out index c38a215..5701676 100644 --- ql/src/test/results/clientpositive/llap/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_limit.q.out @@ -64,17 +64,17 @@ POSTHOOK: query: SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdoubl POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --1887561756 1839.0 -1887561756 -10011.0 -1887561756 -13877.0 --1887561756 10361.0 --1887561756 -8881.0 -1887561756 -2281.0 +-1887561756 -8881.0 +-1887561756 10361.0 +-1887561756 1839.0 -1887561756 9531.0 -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -120,10 +120,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 9173 Data size: 109584 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 value expressions: _col2 (type: smallint) @@ -134,18 +133,29 @@ 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: + dataColumnCount: 12 + includeColumns: [0, 1, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 3 + dataColumns: KEY.reducesinkkey0:tinyint, KEY.reducesinkkey1:double, VALUE._col0:smallint + partitionColumnCount: 0 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: tinyint), KEY.reducesinkkey1 (type: double), VALUE._col0 (type: smallint) @@ -186,10 +196,17 @@ POSTHOOK: query: select ctinyint,cdouble,csmallint from alltypesorc where ctinyi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --64 -15920.0 -15920 -64 -10462.0 -10462 --64 -9842.0 -9842 --64 -8080.0 -8080 +-64 -15920.0 -15920 +-64 -1600.0 -1600 +-64 -200.0 -200 +-64 -2919.0 -2919 +-64 -3097.0 -3097 +-64 -3586.0 -3586 +-64 -4018.0 -4018 +-64 -4040.0 -4040 +-64 -4803.0 -4803 +-64 -6907.0 -6907 -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 @@ -197,19 +214,12 @@ POSTHOOK: Input: default@alltypesorc -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 --64 -6907.0 -6907 --64 -4803.0 -4803 --64 -4040.0 -4040 --64 -4018.0 -4018 --64 -3586.0 -3586 --64 -3097.0 -3097 --64 -2919.0 -2919 --64 -1600.0 -1600 --64 -200.0 -200 -PREHOOK: query: explain vectorization expression +-64 -8080.0 -8080 +-64 -9842.0 -9842 +PREHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -276,6 +286,12 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 + scratchColumnTypeNames: double Reducer 2 Execution mode: llap Reduce Vectorization: @@ -315,30 +331,30 @@ POSTHOOK: query: select ctinyint,avg(cdouble + 1) from alltypesorc group by ctin POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 9370.0945309795 --64 373.52941176470586 --63 2178.7272727272725 --62 245.69387755102042 --61 914.3404255319149 --60 1071.82 --59 318.27272727272725 --58 3483.2444444444445 --57 1867.0535714285713 --56 2595.818181818182 --55 2385.595744680851 --54 2712.7272727272725 --53 -532.7567567567568 --52 2810.705882352941 --51 -96.46341463414635 --50 -960.0192307692307 --49 768.7659574468086 --48 1672.909090909091 --47 -574.6428571428571 -46 3033.55 -PREHOOK: query: explain vectorization expression +-47 -574.6428571428571 +-48 1672.909090909091 +-49 768.7659574468086 +-50 -960.0192307692307 +-51 -96.46341463414635 +-52 2810.705882352941 +-53 -532.7567567567568 +-54 2712.7272727272725 +-55 2385.595744680851 +-56 2595.818181818182 +-57 1867.0535714285713 +-58 3483.2444444444445 +-59 318.27272727272725 +-60 1071.82 +-61 914.3404255319149 +-62 245.69387755102042 +-63 2178.7272727272725 +-64 373.52941176470586 +NULL 9370.0945309795 +PREHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -389,10 +405,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false + className: VectorReduceSinkLongOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 95 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 Execution mode: vectorized, llap @@ -405,15 +420,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: a + reduceColumnSortOrder: + groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 1 + dataColumns: KEY._col0:tinyint + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator Group By Vectorization: @@ -457,30 +483,30 @@ POSTHOOK: query: select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL --64 --63 --62 --61 --60 --59 --58 --57 --56 --55 --54 --53 --52 --51 --50 --49 --48 --47 -46 -PREHOOK: query: explain vectorization expression +-47 +-48 +-49 +-50 +-51 +-52 +-53 +-54 +-55 +-56 +-57 +-58 +-59 +-60 +-61 +-62 +-63 +-64 +NULL +PREHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -533,7 +559,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -545,15 +571,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY._col0:tinyint, KEY._col1:double + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator Group By Vectorization: @@ -610,30 +647,30 @@ POSTHOOK: query: select ctinyint, count(distinct(cdouble)) from alltypesorc grou POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 2932 --64 24 --63 19 --62 27 --61 25 --60 27 --59 31 --58 23 --57 35 --56 36 --55 29 --54 26 --53 22 --52 33 --51 21 --50 30 --49 26 --48 29 --47 22 -46 24 -PREHOOK: query: explain vectorization expression +-47 22 +-48 29 +-49 26 +-50 30 +-51 21 +-52 33 +-53 22 +-54 26 +-55 29 +-56 36 +-57 35 +-58 23 +-59 31 +-60 27 +-61 25 +-62 27 +-63 19 +-64 24 +NULL 2932 +PREHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -658,10 +695,10 @@ POSTHOOK: query: select ctinyint,cdouble from alltypesorc order by ctinyint limi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -696,37 +733,29 @@ STAGE PLANS: predicateExpression: SelectColumnIsNotNull(col 0) -> boolean predicate: ctinyint is not null (type: boolean) Statistics: Num rows: 9173 Data size: 82188 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: cdouble (type: double), ctinyint (type: tinyint) - outputColumnNames: cdouble, ctinyint - Select Vectorization: - className: VectorSelectOperator - native: true - projectedOutputColumns: [5, 0] - Statistics: Num rows: 9173 Data size: 82188 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: sum(ctinyint) - Group By Vectorization: - aggregators: VectorUDAFSumLong(col 0) -> bigint - className: VectorGroupByOperator - vectorOutput: true - keyExpressions: col 5 - native: false - projectedOutputColumns: [0] - keys: cdouble (type: double) - mode: hash - outputColumnNames: _col0, _col1 + Group By Operator + aggregations: sum(ctinyint) + Group By Vectorization: + aggregators: VectorUDAFSumLong(col 0) -> bigint + className: VectorGroupByOperator + vectorOutput: true + keyExpressions: col 5 + native: false + projectedOutputColumns: [0] + keys: cdouble (type: double) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 - Reduce Output Operator - key expressions: _col0 (type: double) - sort order: + - Map-reduce partition columns: _col0 (type: double) - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 3185 Data size: 44512 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: bigint) + value expressions: _col1 (type: bigint) Execution mode: vectorized, llap LLAP IO: all inputs Map Vectorization: @@ -737,15 +766,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized, llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: a + reduceColumnSortOrder: + groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY._col0:double, VALUE._col0:bigint + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator aggregations: sum(VALUE._col0) @@ -764,10 +804,9 @@ STAGE PLANS: key expressions: _col1 (type: bigint), _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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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 TopN Hash Memory Usage: 0.3 Reducer 3 @@ -775,10 +814,16 @@ STAGE PLANS: Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY.reducesinkkey0:bigint, KEY.reducesinkkey1:double + partitionColumnCount: 0 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey1 (type: double), KEY.reducesinkkey0 (type: bigint) @@ -819,23 +864,23 @@ POSTHOOK: query: select cdouble, sum(ctinyint) as sum from alltypesorc where cti POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL -32768 +-10462.0 -64 +-1121.0 -89 +-11322.0 -101 +-11492.0 -78 +-15920.0 -64 +-4803.0 -64 +-6907.0 -64 -7196.0 -2009 +-8080.0 -64 +-8118.0 -80 +-9842.0 -64 +10496.0 -67 15601.0 -1733 -4811.0 -115 --11322.0 -101 --1121.0 -89 -7705.0 -88 3520.0 -86 --8118.0 -80 +4811.0 -115 5241.0 -80 --11492.0 -78 -9452.0 -76 557.0 -75 -10496.0 -67 --15920.0 -64 --10462.0 -64 --9842.0 -64 --8080.0 -64 --6907.0 -64 --4803.0 -64 +7705.0 -88 +9452.0 -76 +NULL -32768 diff --git ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out index f1fe221..bdfe78e 100644 --- ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_offset_limit.q.out @@ -116,10 +116,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 9173 Data size: 109584 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: smallint) @@ -130,7 +129,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_part_project.q.out ql/src/test/results/clientpositive/llap/vectorization_part_project.q.out index 872e7f3..85c4dd0 100644 --- ql/src/test/results/clientpositive/llap/vectorization_part_project.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_part_project.q.out @@ -87,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 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 f0b28fa..c141e36 100644 --- ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out +++ ql/src/test/results/clientpositive/llap/vectorization_short_regress.q.out @@ -1022,10 +1022,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: timestamp), _col3 (type: string), _col4 (type: boolean), _col5 (type: tinyint), _col6 (type: float), _col7 (type: timestamp), _col8 (type: smallint), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: int), _col12 (type: decimal(14,3)), _col13 (type: smallint), _col14 (type: smallint), _col15 (type: smallint), _col16 (type: double), _col17 (type: decimal(15,3)), _col18 (type: float), _col19 (type: double), _col20 (type: double), _col21 (type: tinyint), _col22 (type: decimal(9,7)) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 9898 Data size: 5632662 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -1035,7 +1034,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 @@ -1321,10 +1320,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: boolean), _col4 (type: float), _col5 (type: double), _col6 (type: timestamp), _col7 (type: smallint), _col8 (type: string), _col9 (type: boolean), _col10 (type: double), _col11 (type: decimal(5,3)), _col12 (type: double), _col13 (type: float), _col14 (type: float), _col15 (type: float), _col16 (type: float), _col17 (type: double), _col18 (type: double), _col19 (type: bigint), _col20 (type: double), _col21 (type: smallint), _col22 (type: bigint), _col23 (type: double), _col24 (type: smallint) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 8195 Data size: 3349694 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized, llap @@ -1334,7 +1332,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 @@ -1569,10 +1567,9 @@ STAGE PLANS: key expressions: _col8 (type: boolean), _col1 (type: string), _col3 (type: timestamp), _col5 (type: float), _col6 (type: bigint), _col1 (type: string), _col4 (type: double), _col0 (type: int), _col7 (type: smallint), _col4 (type: double), _col9 (type: int), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: float), _col13 (type: bigint), _col14 (type: double), _col15 (type: double), _col16 (type: bigint), _col17 (type: double), _col18 (type: decimal(8,7)), _col19 (type: double), _col20 (type: smallint), _col21 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 10922 Data size: 3594034 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: boolean) @@ -1583,7 +1580,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 @@ -1875,10 +1872,9 @@ STAGE PLANS: key expressions: _col5 (type: smallint), _col1 (type: string), _col2 (type: double), _col3 (type: float), _col4 (type: bigint), _col6 (type: double), _col7 (type: int), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: float), _col13 (type: int), _col14 (type: double), _col15 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3868 Data size: 748844 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: timestamp) @@ -1889,7 +1885,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 @@ -3286,7 +3282,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -3400,7 +3396,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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,7 +3582,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -3700,7 +3696,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -3814,7 +3810,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -3928,7 +3924,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -4042,7 +4038,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -4156,7 +4152,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 3f0afcb..855d2e8 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -428,7 +428,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 2 Data size: 52 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 940b36e..ab75515 100644 --- ql/src/test/results/clientpositive/llap/vectorized_case.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_case.q.out @@ -301,7 +301,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -428,7 +428,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 c98ea9c..952c82a 100644 --- ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out @@ -1272,7 +1272,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1312,7 +1312,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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_semijoin_reduction.q.out ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out index 5a7a101..8f83ee2 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 57 Data size: 10182 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -147,7 +147,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -240,7 +240,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) @@ -315,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -358,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 57 Data size: 10182 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) @@ -384,7 +384,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -477,7 +477,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) @@ -552,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -595,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 57 Data size: 10182 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) @@ -621,7 +621,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -714,7 +714,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) @@ -790,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -833,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 57 Data size: 10182 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -859,7 +859,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -903,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 57 Data size: 10182 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -929,7 +929,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1024,7 +1024,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1053,7 +1053,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) @@ -1129,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1172,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 57 Data size: 10182 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) @@ -1198,7 +1198,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1225,7 +1225,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1318,7 +1318,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1347,7 +1347,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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) @@ -1422,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -1465,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 29 Data size: 5180 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) @@ -1491,7 +1491,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -1584,7 +1584,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 1c72876..0f02856 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 37eb47e..b96dace 100644 --- ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out @@ -103,7 +103,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -146,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 df4b0d8..6086e03 100644 --- ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_ptf.q.out @@ -156,7 +156,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -372,7 +372,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) Execution mode: vectorized, llap @@ -412,7 +412,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -616,7 +616,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -770,7 +770,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -984,7 +984,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -1201,7 +1201,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -1420,7 +1420,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Execution mode: vectorized, llap @@ -1460,7 +1460,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1634,7 +1634,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1666,7 +1666,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Execution mode: vectorized, llap @@ -2245,7 +2245,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2458,7 +2458,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2741,7 +2741,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2958,7 +2958,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -2998,7 +2998,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -3242,7 +3242,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -3473,7 +3473,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: double) Execution mode: vectorized, llap @@ -3729,7 +3729,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized, llap @@ -4165,7 +4165,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_size (type: int) Execution mode: vectorized, llap @@ -4479,7 +4479,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_size (type: int) Execution mode: vectorized, llap @@ -4789,7 +4789,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -5075,7 +5075,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -5404,7 +5404,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap @@ -5702,7 +5702,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized, llap diff --git ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out index 7f04eba..3c972cc 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 ed509de..5c36a4a 100644 --- ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out @@ -128,7 +128,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -311,7 +311,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -494,7 +494,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -677,7 +677,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 @@ -820,7 +820,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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 cd9dfc7..fa4b51f 100644 --- ql/src/test/results/clientpositive/spark/vector_between_in.q.out +++ ql/src/test/results/clientpositive/spark/vector_between_in.q.out @@ -60,7 +60,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -162,7 +162,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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,7 +260,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -362,7 +362,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -460,7 +460,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -554,7 +554,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -648,7 +648,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -750,7 +750,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1102,7 +1102,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1143,7 +1143,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1238,7 +1238,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1279,7 +1279,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1374,7 +1374,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1415,7 +1415,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1510,7 +1510,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1551,7 +1551,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 4454445..ae46718 100644 --- ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out +++ ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out @@ -1281,7 +1281,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -1329,7 +1329,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 57c1963..3244e47 100644 --- ql/src/test/results/clientpositive/spark/vector_data_types.q.out +++ ql/src/test/results/clientpositive/spark/vector_data_types.q.out @@ -230,10 +230,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 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -243,7 +242,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/vector_decimal_aggregate.q.out ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out index e07cf83..edda919 100644 --- ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out @@ -85,7 +85,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 b7b0603..59dcf7c 100644 --- ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out +++ ql/src/test/results/clientpositive/spark/vector_distinct_2.q.out @@ -156,7 +156,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 e4e9761..94b3ef6 100644 --- ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out +++ ql/src/test/results/clientpositive/spark/vector_groupby_3.q.out @@ -158,7 +158,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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_mapjoin_reduce.q.out ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out index dc80037..2f2609f 100644 --- ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out @@ -157,7 +157,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -428,7 +428,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: 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 e9b9fca..fd3469c 100644 --- ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out @@ -159,7 +159,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -200,7 +200,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 098b898..b9ec98a 100644 --- ql/src/test/results/clientpositive/spark/vector_outer_join1.q.out +++ ql/src/test/results/clientpositive/spark/vector_outer_join1.q.out @@ -828,7 +828,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 f238b0a..9e17983 100644 --- ql/src/test/results/clientpositive/spark/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/spark/vector_outer_join2.q.out @@ -409,7 +409,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 f7fe637..b361ec0 100644 --- ql/src/test/results/clientpositive/spark/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/spark/vector_string_concat.q.out @@ -367,10 +367,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -407,10 +406,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 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 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 eda5612..ae7e0c3 100644 --- ql/src/test/results/clientpositive/spark/vectorization_0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_0.q.out @@ -62,7 +62,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -101,7 +101,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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,7 +216,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -255,7 +255,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -512,7 +512,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -551,7 +551,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -666,7 +666,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -705,7 +705,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -962,7 +962,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1001,7 +1001,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1116,7 +1116,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -1155,7 +1155,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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_7.q.out ql/src/test/results/clientpositive/spark/vectorization_7.q.out index 9783907..d2ff353 100644 --- ql/src/test/results/clientpositive/spark/vectorization_7.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_7.q.out @@ -95,10 +95,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (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, 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 7281 Data size: 223523 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -107,7 +106,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 @@ -329,10 +328,9 @@ STAGE PLANS: key expressions: _col0 (type: boolean), _col1 (type: bigint), _col2 (type: smallint), _col3 (type: tinyint), _col4 (type: timestamp), _col5 (type: string), _col6 (type: bigint), _col7 (type: int), _col8 (type: smallint), _col9 (type: tinyint), _col10 (type: int), _col11 (type: bigint), _col12 (type: int), _col13 (type: tinyint), _col14 (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, 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 7281 Data size: 223523 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -341,7 +339,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_8.q.out ql/src/test/results/clientpositive/spark/vectorization_8.q.out index b5c056f..927ee59 100644 --- ql/src/test/results/clientpositive/spark/vectorization_8.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_8.q.out @@ -91,10 +91,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (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, 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -103,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 @@ -312,10 +311,9 @@ STAGE PLANS: key expressions: _col0 (type: timestamp), _col1 (type: double), _col2 (type: boolean), _col3 (type: string), _col4 (type: float), _col5 (type: double), _col6 (type: double), _col7 (type: double), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: float), _col12 (type: float), _col13 (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, 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 + className: VectorReduceSinkObjectHashOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -324,7 +322,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 81979d6..788c2ee 100644 --- ql/src/test/results/clientpositive/spark/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_div0.q.out @@ -225,10 +225,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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)) @@ -238,7 +237,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 @@ -443,10 +442,9 @@ STAGE PLANS: key expressions: _col0 (type: double), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -456,7 +454,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_part_project.q.out ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out index 361384f..7e1cde0 100644 --- ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out @@ -85,7 +85,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_short_regress.q.out ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out index 6d7c103..cc0f029 100644 --- ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_short_regress.q.out @@ -1009,10 +1009,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: double), _col2 (type: timestamp), _col3 (type: string), _col4 (type: boolean), _col5 (type: tinyint), _col6 (type: float), _col7 (type: timestamp), _col8 (type: smallint), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: int), _col12 (type: decimal(14,3)), _col13 (type: smallint), _col14 (type: smallint), _col15 (type: smallint), _col16 (type: double), _col17 (type: decimal(15,3)), _col18 (type: float), _col19 (type: double), _col20 (type: double), _col21 (type: tinyint), _col22 (type: decimal(9,7)) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 9898 Data size: 303864 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -1021,7 +1020,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 @@ -1306,10 +1305,9 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: boolean), _col4 (type: float), _col5 (type: double), _col6 (type: timestamp), _col7 (type: smallint), _col8 (type: string), _col9 (type: boolean), _col10 (type: double), _col11 (type: decimal(5,3)), _col12 (type: double), _col13 (type: float), _col14 (type: float), _col15 (type: float), _col16 (type: float), _col17 (type: double), _col18 (type: double), _col19 (type: bigint), _col20 (type: double), _col21 (type: smallint), _col22 (type: bigint), _col23 (type: double), _col24 (type: smallint) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -1318,7 +1316,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 @@ -1552,10 +1550,9 @@ STAGE PLANS: key expressions: _col8 (type: boolean), _col1 (type: string), _col3 (type: timestamp), _col5 (type: float), _col6 (type: bigint), _col1 (type: string), _col4 (type: double), _col0 (type: int), _col7 (type: smallint), _col4 (type: double), _col9 (type: int), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: float), _col13 (type: bigint), _col14 (type: double), _col15 (type: double), _col16 (type: bigint), _col17 (type: double), _col18 (type: decimal(8,7)), _col19 (type: double), _col20 (type: smallint), _col21 (type: int) 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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 10922 Data size: 335301 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col2 (type: boolean) @@ -1565,7 +1562,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 @@ -1856,10 +1853,9 @@ STAGE PLANS: key expressions: _col5 (type: smallint), _col1 (type: string), _col2 (type: double), _col3 (type: float), _col4 (type: bigint), _col6 (type: double), _col7 (type: int), _col8 (type: float), _col9 (type: double), _col10 (type: double), _col11 (type: double), _col12 (type: float), _col13 (type: int), _col14 (type: double), _col15 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 3868 Data size: 118746 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col0 (type: timestamp) @@ -1869,7 +1865,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 @@ -3253,7 +3249,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -3365,7 +3361,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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,7 +3545,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -3661,7 +3657,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -3773,7 +3769,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -3885,7 +3881,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -3997,7 +3993,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -4109,7 +4105,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 ead71ad..1096c90 100644 --- ql/src/test/results/clientpositive/spark/vectorized_case.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_case.q.out @@ -298,7 +298,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -423,7 +423,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 3e6a73f..96ff8f7 100644 --- ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out @@ -155,7 +155,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -338,7 +338,8 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 - Spark + Tez +#### A masked pattern was here #### Edges: Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2) @@ -365,9 +366,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkLongOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) Execution mode: vectorized @@ -404,9 +405,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkLongOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -423,11 +424,7 @@ STAGE PLANS: dataColumns: p_partkey:int, p_name:string, p_mfgr:string, p_brand:string, p_type:string, p_size:int, p_container:string, p_retailprice:double, p_comment:string partitionColumnCount: 0 Reducer 2 - Reduce Vectorization: - enabled: true - enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true - notVectorizedReason: Tagging not supported - vectorized: false + Execution mode: llap Reduce Operator Tree: Join Operator condition map: @@ -610,7 +607,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -761,7 +758,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -971,7 +968,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -1184,7 +1181,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -1399,7 +1396,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Execution mode: vectorized @@ -1436,9 +1433,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkLongOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -1489,11 +1486,7 @@ STAGE PLANS: Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) Reducer 3 - Reduce Vectorization: - enabled: true - enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true - notVectorizedReason: Tagging not supported - vectorized: false + Execution mode: llap Reduce Operator Tree: Join Operator condition map: @@ -1611,9 +1604,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkLongOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -1644,7 +1637,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) Execution mode: vectorized @@ -2214,7 +2207,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -2423,7 +2416,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -2701,7 +2694,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -2914,7 +2907,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_partkey (type: int), p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -2951,9 +2944,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_partkey (type: int) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkLongOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Map Vectorization: @@ -3197,7 +3190,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -3425,7 +3418,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: double) Execution mode: vectorized @@ -3677,7 +3670,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int), p_retailprice (type: double) Execution mode: vectorized @@ -4123,9 +4116,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkStringOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_size (type: int) Execution mode: vectorized @@ -4432,9 +4425,9 @@ STAGE PLANS: sort order: + Map-reduce partition columns: p_mfgr (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkStringOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_name (type: string), p_size (type: int) Execution mode: vectorized @@ -4736,9 +4729,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkMultiKeyOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -5017,9 +5010,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkMultiKeyOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -5340,9 +5333,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkMultiKeyOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized @@ -5633,9 +5626,9 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator + className: VectorReduceSinkMultiKeyOperator 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE value expressions: p_size (type: int) Execution mode: vectorized diff --git ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out index adf6e65..b6e7519 100644 --- ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_shufflejoin.q.out @@ -53,7 +53,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: @@ -94,7 +94,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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: 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 3892165..fe23597 100644 --- ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out @@ -127,7 +127,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -308,7 +308,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -489,7 +489,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -670,7 +670,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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 @@ -811,7 +811,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, No PTF 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_non_string_partition.q.out ql/src/test/results/clientpositive/tez/vector_non_string_partition.q.out index 647a095..e09cbb9 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 @@ -74,10 +74,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1024 Data size: 4096 Basic stats: COMPLETE Column stats: PARTIAL TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: tinyint) @@ -87,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 @@ -198,10 +197,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 1024 Data size: 121205 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -210,7 +208,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/tez/vectorization_div0.q.out ql/src/test/results/clientpositive/tez/vectorization_div0.q.out index cb7dc90..bcf315e 100644 --- ql/src/test/results/clientpositive/tez/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/tez/vectorization_div0.q.out @@ -226,10 +226,9 @@ STAGE PLANS: key expressions: _col0 (type: bigint), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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)) @@ -239,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 @@ -445,10 +444,9 @@ STAGE PLANS: key expressions: _col0 (type: double), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true 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) @@ -458,7 +456,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/tez/vectorization_limit.q.out ql/src/test/results/clientpositive/tez/vectorization_limit.q.out index 2993eb9..b1d6c1c 100644 --- ql/src/test/results/clientpositive/tez/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/tez/vectorization_limit.q.out @@ -63,17 +63,17 @@ POSTHOOK: query: SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdoubl POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --1887561756 1839.0 -1887561756 -10011.0 -1887561756 -13877.0 --1887561756 10361.0 --1887561756 -8881.0 -1887561756 -2281.0 +-1887561756 -8881.0 +-1887561756 10361.0 +-1887561756 1839.0 -1887561756 9531.0 -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -119,10 +119,9 @@ STAGE PLANS: key expressions: _col0 (type: tinyint), _col1 (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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 9173 Data size: 109584 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 value expressions: _col2 (type: smallint) @@ -132,18 +131,29 @@ 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: + dataColumnCount: 12 + includeColumns: [0, 1, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 3 + dataColumns: KEY.reducesinkkey0:tinyint, KEY.reducesinkkey1:double, VALUE._col0:smallint + partitionColumnCount: 0 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: tinyint), KEY.reducesinkkey1 (type: double), VALUE._col0 (type: smallint) @@ -184,10 +194,17 @@ POSTHOOK: query: select ctinyint,cdouble,csmallint from alltypesorc where ctinyi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --64 -15920.0 -15920 -64 -10462.0 -10462 --64 -9842.0 -9842 --64 -8080.0 -8080 +-64 -15920.0 -15920 +-64 -1600.0 -1600 +-64 -200.0 -200 +-64 -2919.0 -2919 +-64 -3097.0 -3097 +-64 -3586.0 -3586 +-64 -4018.0 -4018 +-64 -4040.0 -4040 +-64 -4803.0 -4803 +-64 -6907.0 -6907 -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 @@ -195,19 +212,12 @@ POSTHOOK: Input: default@alltypesorc -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 --64 -6907.0 -6907 --64 -4803.0 -4803 --64 -4040.0 -4040 --64 -4018.0 -4018 --64 -3586.0 -3586 --64 -3097.0 -3097 --64 -2919.0 -2919 --64 -1600.0 -1600 --64 -200.0 -200 -PREHOOK: query: explain vectorization expression +-64 -8080.0 -8080 +-64 -9842.0 -9842 +PREHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -273,6 +283,12 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 + scratchColumnTypeNames: double Reducer 2 Reduce Vectorization: enabled: true @@ -311,30 +327,30 @@ POSTHOOK: query: select ctinyint,avg(cdouble + 1) from alltypesorc group by ctin POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 9370.0945309795 --64 373.52941176470586 --63 2178.7272727272725 --62 245.69387755102042 --61 914.3404255319149 --60 1071.82 --59 318.27272727272725 --58 3483.2444444444445 --57 1867.0535714285713 --56 2595.818181818182 --55 2385.595744680851 --54 2712.7272727272725 --53 -532.7567567567568 --52 2810.705882352941 --51 -96.46341463414635 --50 -960.0192307692307 --49 768.7659574468086 --48 1672.909090909091 --47 -574.6428571428571 -46 3033.55 -PREHOOK: query: explain vectorization expression +-47 -574.6428571428571 +-48 1672.909090909091 +-49 768.7659574468086 +-50 -960.0192307692307 +-51 -96.46341463414635 +-52 2810.705882352941 +-53 -532.7567567567568 +-54 2712.7272727272725 +-55 2385.595744680851 +-56 2595.818181818182 +-57 1867.0535714285713 +-58 3483.2444444444445 +-59 318.27272727272725 +-60 1071.82 +-61 914.3404255319149 +-62 245.69387755102042 +-63 2178.7272727272725 +-64 373.52941176470586 +NULL 9370.0945309795 +PREHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -385,10 +401,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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN IS false + className: VectorReduceSinkLongOperator + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true Statistics: Num rows: 95 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.3 Execution mode: vectorized @@ -400,15 +415,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: a + reduceColumnSortOrder: + groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 1 + dataColumns: KEY._col0:tinyint + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator Group By Vectorization: @@ -452,30 +478,30 @@ POSTHOOK: query: select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL --64 --63 --62 --61 --60 --59 --58 --57 --56 --55 --54 --53 --52 --51 --50 --49 --48 --47 -46 -PREHOOK: query: explain vectorization expression +-47 +-48 +-49 +-50 +-51 +-52 +-53 +-54 +-55 +-56 +-57 +-58 +-59 +-60 +-61 +-62 +-63 +-64 +NULL +PREHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -528,7 +554,7 @@ STAGE PLANS: Reduce Sink Vectorization: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF 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: @@ -539,15 +565,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY._col0:tinyint, KEY._col1:double + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator Group By Vectorization: @@ -604,30 +641,30 @@ POSTHOOK: query: select ctinyint, count(distinct(cdouble)) from alltypesorc grou POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 2932 --64 24 --63 19 --62 27 --61 25 --60 27 --59 31 --58 23 --57 35 --56 36 --55 29 --54 26 --53 22 --52 33 --51 21 --50 30 --49 26 --48 29 --47 22 -46 24 -PREHOOK: query: explain vectorization expression +-47 22 +-48 29 +-49 26 +-50 30 +-51 21 +-52 33 +-53 22 +-54 26 +-55 29 +-56 36 +-57 35 +-58 23 +-59 31 +-60 27 +-61 25 +-62 27 +-63 19 +-64 24 +NULL 2932 +PREHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -652,10 +689,10 @@ POSTHOOK: query: select ctinyint,cdouble from alltypesorc order by ctinyint limi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -710,7 +747,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 TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS 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 PTF 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 @@ -722,15 +759,26 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reducer 2 Execution mode: vectorized Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: a + reduceColumnSortOrder: + groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY._col0:double, VALUE._col0:bigint + partitionColumnCount: 0 Reduce Operator Tree: Group By Operator aggregations: sum(VALUE._col0) @@ -749,10 +797,9 @@ STAGE PLANS: key expressions: _col1 (type: bigint), _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 DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - nativeConditionsNotMet: No TopN 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 PTF 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 TopN Hash Memory Usage: 0.3 Reducer 3 @@ -760,10 +807,16 @@ STAGE PLANS: Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true + reduceColumnNullOrder: aa + reduceColumnSortOrder: ++ groupByVectorOutput: true allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + dataColumns: KEY.reducesinkkey0:bigint, KEY.reducesinkkey1:double + partitionColumnCount: 0 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey1 (type: double), KEY.reducesinkkey0 (type: bigint) @@ -804,23 +857,23 @@ POSTHOOK: query: select cdouble, sum(ctinyint) as sum from alltypesorc where cti POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL -32768 +-10462.0 -64 +-1121.0 -89 +-11322.0 -101 +-11492.0 -78 +-15920.0 -64 +-4803.0 -64 +-6907.0 -64 -7196.0 -2009 +-8080.0 -64 +-8118.0 -80 +-9842.0 -64 +10496.0 -67 15601.0 -1733 -4811.0 -115 --11322.0 -101 --1121.0 -89 -7705.0 -88 3520.0 -86 --8118.0 -80 +4811.0 -115 5241.0 -80 --11492.0 -78 -9452.0 -76 557.0 -75 -10496.0 -67 --15920.0 -64 --10462.0 -64 --9842.0 -64 --8080.0 -64 --6907.0 -64 --4803.0 -64 +7705.0 -88 +9452.0 -76 +NULL -32768 diff --git ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out index 6422bd5..3519a87 100644 --- ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out +++ ql/src/test/results/clientpositive/vector_binary_join_groupby.q.out @@ -201,7 +201,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -250,7 +250,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 Execution mode: vectorized @@ -357,7 +357,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -409,7 +409,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 14819 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) diff --git ql/src/test/results/clientpositive/vector_bucket.q.out ql/src/test/results/clientpositive/vector_bucket.q.out index 0eeb8a5..3b74023 100644 --- ql/src/test/results/clientpositive/vector_bucket.q.out +++ ql/src/test/results/clientpositive/vector_bucket.q.out @@ -45,7 +45,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 26 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string) diff --git ql/src/test/results/clientpositive/vector_cast_constant.q.out ql/src/test/results/clientpositive/vector_cast_constant.q.out index c323add..7afdb72 100644 --- ql/src/test/results/clientpositive/vector_cast_constant.q.out +++ ql/src/test/results/clientpositive/vector_cast_constant.q.out @@ -201,8 +201,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: decimal(14,4)) diff --git ql/src/test/results/clientpositive/vector_char_2.q.out ql/src/test/results/clientpositive/vector_char_2.q.out index d4e5225..03bf436 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -156,8 +156,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 250 Data size: 49500 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -298,8 +298,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 99000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -347,8 +347,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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/vector_char_mapjoin1.q.out ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out index d1bdadb..3b022d9 100644 --- ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out +++ ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out @@ -204,7 +204,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) @@ -335,7 +335,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) @@ -468,7 +468,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_char_simple.q.out ql/src/test/results/clientpositive/vector_char_simple.q.out index c7b00fd..72ea17b 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 f158236..87ab937 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 6778499..431cfdc 100644 --- ql/src/test/results/clientpositive/vector_coalesce_2.q.out +++ ql/src/test/results/clientpositive/vector_coalesce_2.q.out @@ -220,7 +220,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_count.q.out ql/src/test/results/clientpositive/vector_count.q.out index 3473759..ff6993e 100644 --- ql/src/test/results/clientpositive/vector_count.q.out +++ ql/src/test/results/clientpositive/vector_count.q.out @@ -93,7 +93,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -196,7 +196,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -283,7 +283,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -373,7 +373,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_data_types.q.out ql/src/test/results/clientpositive/vector_data_types.q.out index b0bdeb1..f6d20ae 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 8268869..34c60c0 100644 --- ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out @@ -80,7 +80,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_decimal_expressions.q.out ql/src/test/results/clientpositive/vector_decimal_expressions.q.out index 627acfd..56d2e1f 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 de49c17..d92b6c2 100644 --- ql/src/test/results/clientpositive/vector_decimal_round.q.out +++ ql/src/test/results/clientpositive/vector_decimal_round.q.out @@ -67,7 +67,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 3 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: decimal(11,0)) @@ -150,7 +150,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 3 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: decimal(10,0)) @@ -418,7 +418,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) @@ -501,7 +501,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) 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 5d1d05e..535448a 100644 --- ql/src/test/results/clientpositive/vector_decimal_round_2.q.out +++ ql/src/test/results/clientpositive/vector_decimal_round_2.q.out @@ -71,7 +71,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) @@ -204,7 +204,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) @@ -364,7 +364,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) @@ -513,7 +513,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)) diff --git ql/src/test/results/clientpositive/vector_distinct_2.q.out ql/src/test/results/clientpositive/vector_distinct_2.q.out index 4bf3f0f..b6e9527 100644 --- ql/src/test/results/clientpositive/vector_distinct_2.q.out +++ ql/src/test/results/clientpositive/vector_distinct_2.q.out @@ -151,7 +151,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_empty_where.q.out ql/src/test/results/clientpositive/vector_empty_where.q.out index a1fb19d..b2dec6d 100644 --- ql/src/test/results/clientpositive/vector_empty_where.q.out +++ ql/src/test/results/clientpositive/vector_empty_where.q.out @@ -56,7 +56,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -155,7 +155,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -262,7 +262,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -369,7 +369,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No TopN IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_groupby4.q.out ql/src/test/results/clientpositive/vector_groupby4.q.out index 799797d..9de8e6e 100644 --- ql/src/test/results/clientpositive/vector_groupby4.q.out +++ ql/src/test/results/clientpositive/vector_groupby4.q.out @@ -62,7 +62,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -109,7 +109,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_groupby6.q.out ql/src/test/results/clientpositive/vector_groupby6.q.out index 6fee467..25cf5b2 100644 --- ql/src/test/results/clientpositive/vector_groupby6.q.out +++ ql/src/test/results/clientpositive/vector_groupby6.q.out @@ -62,7 +62,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -109,7 +109,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_groupby_3.q.out ql/src/test/results/clientpositive/vector_groupby_3.q.out index f4be253..9a1256b 100644 --- ql/src/test/results/clientpositive/vector_groupby_3.q.out +++ ql/src/test/results/clientpositive/vector_groupby_3.q.out @@ -153,7 +153,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out index df1d435..addbdeb 100644 --- ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out +++ ql/src/test/results/clientpositive/vector_groupby_mapjoin.q.out @@ -63,7 +63,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -244,7 +244,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 366 Data size: 10110 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) @@ -349,7 +349,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 5312 Basic stats: COMPLETE Column stats: NONE 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 43ce596..bc59510 100644 --- ql/src/test/results/clientpositive/vector_groupby_reduce.q.out +++ ql/src/test/results/clientpositive/vector_groupby_reduce.q.out @@ -274,8 +274,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -321,8 +321,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 44138 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -469,7 +469,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -529,7 +529,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 250 Data size: 22069 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -743,7 +743,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -805,7 +805,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 250 Data size: 22069 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: bigint) @@ -1020,7 +1020,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -1082,7 +1082,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 250 Data size: 22069 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: bigint) diff --git ql/src/test/results/clientpositive/vector_grouping_sets.q.out ql/src/test/results/clientpositive/vector_grouping_sets.q.out index a4199ed..3d35fbf 100644 --- ql/src/test/results/clientpositive/vector_grouping_sets.q.out +++ ql/src/test/results/clientpositive/vector_grouping_sets.q.out @@ -177,7 +177,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 @@ -291,7 +291,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_if_expr.q.out ql/src/test/results/clientpositive/vector_if_expr.q.out index 043940b..2f1cf0a 100644 --- ql/src/test/results/clientpositive/vector_if_expr.q.out +++ ql/src/test/results/clientpositive/vector_if_expr.q.out @@ -44,7 +44,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) 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 e5e3a52..8c8ef80 100644 --- ql/src/test/results/clientpositive/vector_include_no_sel.q.out +++ ql/src/test/results/clientpositive/vector_include_no_sel.q.out @@ -250,7 +250,7 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: false - nativeConditionsMet: No TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + nativeConditionsMet: No PTF 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) diff --git ql/src/test/results/clientpositive/vector_interval_1.q.out ql/src/test/results/clientpositive/vector_interval_1.q.out index f53a2c2..2a398ae 100644 --- ql/src/test/results/clientpositive/vector_interval_1.q.out +++ ql/src/test/results/clientpositive/vector_interval_1.q.out @@ -85,7 +85,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -194,7 +194,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -311,7 +311,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -440,7 +440,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -581,7 +581,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -704,7 +704,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -809,7 +809,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -920,7 +920,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out index 75250e3..b67231c 100644 --- ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out +++ ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out @@ -93,7 +93,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -258,7 +258,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -423,7 +423,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -583,8 +583,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 800 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -701,7 +701,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -868,7 +868,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -1035,7 +1035,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out index 3e96d10..82bef24 100644 --- ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out @@ -55,7 +55,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 11999 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -288,7 +288,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 11999 Basic stats: COMPLETE Column stats: NONE 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 018a193..1d13a65 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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_order_null.q.out ql/src/test/results/clientpositive/vector_order_null.q.out index ca0ea30..d65b3ec 100644 --- ql/src/test/results/clientpositive/vector_order_null.q.out +++ ql/src/test/results/clientpositive/vector_order_null.q.out @@ -99,7 +99,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -193,7 +193,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -287,7 +287,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -381,7 +381,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -475,7 +475,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -569,7 +569,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -663,7 +663,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -757,7 +757,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -851,7 +851,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -945,7 +945,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized @@ -1039,7 +1039,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6 Data size: 22 Basic stats: COMPLETE Column stats: NONE 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 7d27526..b85eb75 100644 --- ql/src/test/results/clientpositive/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/vector_orderby_5.q.out @@ -154,7 +154,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -202,7 +202,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 459356 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) diff --git ql/src/test/results/clientpositive/vector_outer_join1.q.out ql/src/test/results/clientpositive/vector_outer_join1.q.out index 87ff0c4..dc2010d 100644 --- ql/src/test/results/clientpositive/vector_outer_join1.q.out +++ ql/src/test/results/clientpositive/vector_outer_join1.q.out @@ -699,7 +699,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_outer_join2.q.out ql/src/test/results/clientpositive/vector_outer_join2.q.out index f8e757e..2d2c2b5 100644 --- ql/src/test/results/clientpositive/vector_outer_join2.q.out +++ ql/src/test/results/clientpositive/vector_outer_join2.q.out @@ -346,7 +346,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vector_outer_join3.q.out ql/src/test/results/clientpositive/vector_outer_join3.q.out index d97ff1d..9b93644 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_1","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_23","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_22","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_13","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_14","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_1","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_23","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_22","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_13","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No PTF 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_14","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_1","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_23","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_22","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_13","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_14","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_1","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_23","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_22","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_13","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No PTF 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_14","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_1","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_23","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_22","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_13","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_14","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_1","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_23","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_22","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_13","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No PTF 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_14","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 6ea8237..226898a 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_1","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_23","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_22","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_13","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_14","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_1","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_23","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_22","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_13","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","No PTF 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_14","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 2a985b1..68f836d 100644 --- ql/src/test/results/clientpositive/vector_reduce1.q.out +++ ql/src/test/results/clientpositive/vector_reduce1.q.out @@ -139,7 +139,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_reduce2.q.out ql/src/test/results/clientpositive/vector_reduce2.q.out index 27ea4ff..0da1f5c 100644 --- ql/src/test/results/clientpositive/vector_reduce2.q.out +++ ql/src/test/results/clientpositive/vector_reduce2.q.out @@ -139,7 +139,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 diff --git ql/src/test/results/clientpositive/vector_reduce3.q.out ql/src/test/results/clientpositive/vector_reduce3.q.out index 2264366..8c20fe8 100644 --- ql/src/test/results/clientpositive/vector_reduce3.q.out +++ ql/src/test/results/clientpositive/vector_reduce3.q.out @@ -139,7 +139,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 73aa28b..724ef45 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 6102 Data size: 1440072 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 value expressions: _col4 (type: decimal(20,10)) @@ -123,8 +123,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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/vector_string_concat.q.out ql/src/test/results/clientpositive/vector_string_concat.q.out index 3451a7e..00f9b38 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -408,8 +408,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 459356 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vector_tablesample_rows.q.out ql/src/test/results/clientpositive/vector_tablesample_rows.q.out index 38f13da..c96ea00 100644 --- ql/src/test/results/clientpositive/vector_tablesample_rows.q.out +++ ql/src/test/results/clientpositive/vector_tablesample_rows.q.out @@ -281,7 +281,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: COMPLETE value expressions: _col0 (type: bigint) diff --git ql/src/test/results/clientpositive/vector_varchar_simple.q.out ql/src/test/results/clientpositive/vector_varchar_simple.q.out index 1bed203..0f8bdb5 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 7476bbb..a7ab4ef 100644 --- ql/src/test/results/clientpositive/vector_when_case_null.q.out +++ ql/src/test/results/clientpositive/vector_when_case_null.q.out @@ -67,7 +67,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vectorization_13.q.out ql/src/test/results/clientpositive/vectorization_13.q.out index 77ca3a2..35c704e 100644 --- ql/src/test/results/clientpositive/vectorization_13.q.out +++ ql/src/test/results/clientpositive/vectorization_13.q.out @@ -166,8 +166,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized @@ -481,8 +481,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 1365 Data size: 293479 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 Execution mode: vectorized diff --git ql/src/test/results/clientpositive/vectorization_7.q.out ql/src/test/results/clientpositive/vectorization_7.q.out index e534296..c05fee0 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 7650c5f..ce2a4b5 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 83dff79..1dd7180 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 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 fc774f0..7381294 100644 --- ql/src/test/results/clientpositive/vectorization_limit.q.out +++ ql/src/test/results/clientpositive/vectorization_limit.q.out @@ -60,17 +60,17 @@ POSTHOOK: query: SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdoubl POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --1887561756 1839.0 -1887561756 -10011.0 -1887561756 -13877.0 --1887561756 10361.0 --1887561756 -8881.0 -1887561756 -2281.0 +-1887561756 -8881.0 +-1887561756 10361.0 +-1887561756 1839.0 -1887561756 9531.0 -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -112,8 +112,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 TopN Hash Memory Usage: 0.3 value expressions: _col2 (type: smallint) @@ -126,6 +126,11 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 1, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reduce Vectorization: enabled: false enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true @@ -160,10 +165,17 @@ POSTHOOK: query: select ctinyint,cdouble,csmallint from alltypesorc where ctinyi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### --64 -15920.0 -15920 -64 -10462.0 -10462 --64 -9842.0 -9842 --64 -8080.0 -8080 +-64 -15920.0 -15920 +-64 -1600.0 -1600 +-64 -200.0 -200 +-64 -2919.0 -2919 +-64 -3097.0 -3097 +-64 -3586.0 -3586 +-64 -4018.0 -4018 +-64 -4040.0 -4040 +-64 -4803.0 -4803 +-64 -6907.0 -6907 -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 @@ -171,19 +183,12 @@ POSTHOOK: Input: default@alltypesorc -64 -7196.0 -7196 -64 -7196.0 -7196 -64 -7196.0 -7196 --64 -6907.0 -6907 --64 -4803.0 -4803 --64 -4040.0 -4040 --64 -4018.0 -4018 --64 -3586.0 -3586 --64 -3097.0 -3097 --64 -2919.0 -2919 --64 -1600.0 -1600 --64 -200.0 -200 -PREHOOK: query: explain vectorization expression +-64 -8080.0 -8080 +-64 -9842.0 -9842 +PREHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -243,6 +248,12 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 + scratchColumnTypeNames: double Reduce Vectorization: enabled: false enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true @@ -283,30 +294,30 @@ POSTHOOK: query: select ctinyint,avg(cdouble + 1) from alltypesorc group by ctin POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 9370.0945309795 --64 373.52941176470586 --63 2178.7272727272725 --62 245.69387755102042 --61 914.3404255319149 --60 1071.82 --59 318.27272727272725 --58 3483.2444444444445 --57 1867.0535714285713 --56 2595.818181818182 --55 2385.595744680851 --54 2712.7272727272725 --53 -532.7567567567568 --52 2810.705882352941 --51 -96.46341463414635 --50 -960.0192307692307 --49 768.7659574468086 --48 1672.909090909091 --47 -574.6428571428571 -46 3033.55 -PREHOOK: query: explain vectorization expression +-47 -574.6428571428571 +-48 1672.909090909091 +-49 768.7659574468086 +-50 -960.0192307692307 +-51 -96.46341463414635 +-52 2810.705882352941 +-53 -532.7567567567568 +-54 2712.7272727272725 +-55 2385.595744680851 +-56 2595.818181818182 +-57 1867.0535714285713 +-58 3483.2444444444445 +-59 318.27272727272725 +-60 1071.82 +-61 914.3404255319149 +-62 245.69387755102042 +-63 2178.7272727272725 +-64 373.52941176470586 +NULL 9370.0945309795 +PREHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -353,8 +364,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 TopN Hash Memory Usage: 0.3 Execution mode: vectorized @@ -366,6 +377,11 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reduce Vectorization: enabled: false enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true @@ -405,30 +421,30 @@ POSTHOOK: query: select distinct(ctinyint) from alltypesorc limit 20 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL --64 --63 --62 --61 --60 --59 --58 --57 --56 --55 --54 --53 --52 --51 --50 --49 --48 --47 -46 -PREHOOK: query: explain vectorization expression +-47 +-48 +-49 +-50 +-51 +-52 +-53 +-54 +-55 +-56 +-57 +-58 +-59 +-60 +-61 +-62 +-63 +-64 +NULL +PREHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint order by ctinyint limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -477,8 +493,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.3 Execution mode: vectorized @@ -490,6 +506,11 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reduce Vectorization: enabled: false enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true @@ -530,30 +551,30 @@ POSTHOOK: query: select ctinyint, count(distinct(cdouble)) from alltypesorc grou POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL 2932 --64 24 --63 19 --62 27 --61 25 --60 27 --59 31 --58 23 --57 35 --56 36 --55 29 --54 26 --53 22 --52 33 --51 21 --50 30 --49 26 --48 29 --47 22 -46 24 -PREHOOK: query: explain vectorization expression +-47 22 +-48 29 +-49 26 +-50 30 +-51 21 +-52 33 +-53 22 +-54 26 +-55 29 +-56 36 +-57 35 +-58 23 +-59 31 +-60 27 +-61 25 +-62 27 +-63 19 +-64 24 +NULL 2932 +PREHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select ctinyint,cdouble from alltypesorc order by ctinyint limit 0 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -578,10 +599,10 @@ POSTHOOK: query: select ctinyint,cdouble from alltypesorc order by ctinyint limi POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -PREHOOK: query: explain vectorization expression +PREHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 PREHOOK: type: QUERY -POSTHOOK: query: explain vectorization expression +POSTHOOK: query: explain vectorization detail select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20 POSTHOOK: type: QUERY PLAN VECTORIZATION: @@ -630,7 +651,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -643,6 +664,11 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 12 + includeColumns: [0, 5] + dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean + partitionColumnCount: 0 Reduce Vectorization: enabled: false enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true @@ -678,8 +704,8 @@ STAGE PLANS: Reduce Sink Vectorization: className: VectorReduceSinkOperator native: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 TopN Hash Memory Usage: 0.3 Execution mode: vectorized @@ -691,6 +717,11 @@ STAGE PLANS: allNative: false usesVectorUDFAdaptor: false vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: _col0:double, _col1:bigint + partitionColumnCount: 0 Reduce Vectorization: enabled: false enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true @@ -725,23 +756,23 @@ POSTHOOK: query: select cdouble, sum(ctinyint) as sum from alltypesorc where cti POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### -NULL -32768 +-10462.0 -64 +-1121.0 -89 +-11322.0 -101 +-11492.0 -78 +-15920.0 -64 +-4803.0 -64 +-6907.0 -64 -7196.0 -2009 +-8080.0 -64 +-8118.0 -80 +-9842.0 -64 +10496.0 -67 15601.0 -1733 -4811.0 -115 --11322.0 -101 --1121.0 -89 -7705.0 -88 3520.0 -86 --8118.0 -80 +4811.0 -115 5241.0 -80 --11492.0 -78 -9452.0 -76 557.0 -75 -10496.0 -67 --15920.0 -64 --10462.0 -64 --9842.0 -64 --8080.0 -64 --6907.0 -64 --4803.0 -64 +7705.0 -88 +9452.0 -76 +NULL -32768 diff --git ql/src/test/results/clientpositive/vectorization_offset_limit.q.out ql/src/test/results/clientpositive/vectorization_offset_limit.q.out index 5da2ad1..b7442d4 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 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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 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 2be619a..5a7a8a2 100644 --- ql/src/test/results/clientpositive/vectorized_case.q.out +++ ql/src/test/results/clientpositive/vectorized_case.q.out @@ -287,7 +287,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -398,7 +398,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vectorized_date_funcs.q.out ql/src/test/results/clientpositive/vectorized_date_funcs.q.out index 35574f4..4248d08 100644 --- ql/src/test/results/clientpositive/vectorized_date_funcs.q.out +++ ql/src/test/results/clientpositive/vectorized_date_funcs.q.out @@ -1250,7 +1250,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -1297,7 +1297,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint) diff --git ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out index 5e0f699..52aa05b 100644 --- ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out +++ ql/src/test/results/clientpositive/vectorized_mapjoin2.q.out @@ -119,7 +119,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vectorized_parquet_types.q.out ql/src/test/results/clientpositive/vectorized_parquet_types.q.out index 23b910b..46e51f7 100644 --- ql/src/test/results/clientpositive/vectorized_parquet_types.q.out +++ ql/src/test/results/clientpositive/vectorized_parquet_types.q.out @@ -412,7 +412,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 11 Data size: 121 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: double), _col6 (type: decimal(4,2)) diff --git ql/src/test/results/clientpositive/vectorized_shufflejoin.q.out ql/src/test/results/clientpositive/vectorized_shufflejoin.q.out index 4c3093d..d42369f 100644 --- ql/src/test/results/clientpositive/vectorized_shufflejoin.q.out +++ ql/src/test/results/clientpositive/vectorized_shufflejoin.q.out @@ -134,7 +134,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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: 92 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: double) diff --git ql/src/test/results/clientpositive/vectorized_timestamp.q.out ql/src/test/results/clientpositive/vectorized_timestamp.q.out index df7ee50..df8297c 100644 --- ql/src/test/results/clientpositive/vectorized_timestamp.q.out +++ ql/src/test/results/clientpositive/vectorized_timestamp.q.out @@ -308,7 +308,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) diff --git ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out index aaee6e7..a4536fd 100644 --- ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out @@ -122,7 +122,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -287,7 +287,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -452,7 +452,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -617,7 +617,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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) @@ -742,7 +742,7 @@ STAGE PLANS: 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 + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF 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)