http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
new file mode 100644
index 0000000..c52ca19
--- /dev/null
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
@@ -0,0 +1,444 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.LongWritable;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+public class TestVectorIfStatement {
+
+  @Test
+  public void testBoolean() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "boolean");
+  }
+
+  @Test
+  public void testInt() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "int");
+  }
+
+  @Test
+  public void testBigInt() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "bigint");
+  }
+
+  @Test
+  public void testString() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "string");
+  }
+
+  @Test
+  public void testTimestamp() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "timestamp");
+  }
+
+  @Test
+  public void testDate() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "date");
+  }
+
+  @Test
+  public void testIntervalDayTime() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "interval_day_time");
+  }
+
+  @Test
+  public void testIntervalYearMonth() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "interval_year_month");
+  }
+
+  @Test
+  public void testDouble() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "double");
+  }
+
+  @Test
+  public void testChar() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "char(10)");
+  }
+
+  @Test
+  public void testVarchar() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "varchar(15)");
+  }
+
+  @Test
+  public void testBinary() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "binary");
+  }
+
+  @Test
+  public void testDecimalLarge() throws Exception {
+    Random random = new Random(9300);
+
+    doIfTests(random, "decimal(20,8)");
+  }
+
+  @Test
+  public void testDecimalSmall() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTests(random, "decimal(10,4)");
+  }
+
+  public enum IfStmtTestMode {
+    ROW_MODE,
+    ADAPTOR_WHEN,
+    VECTOR_EXPRESSION;
+
+    static final int count = values().length;
+  }
+
+  public enum ColumnScalarMode {
+    COLUMN_COLUMN,
+    COLUMN_SCALAR,
+    SCALAR_COLUMN,
+    SCALAR_SCALAR;
+
+    static final int count = values().length;
+  }
+
+  private void doIfTests(Random random, String typeName)
+      throws Exception {
+    doIfTests(random, typeName, DataTypePhysicalVariation.NONE);
+  }
+
+  private void doIfTests(Random random, String typeName,
+      DataTypePhysicalVariation dataTypePhysicalVariation)
+          throws Exception {
+    for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
+      doIfTestsWithDiffColumnScalar(
+          random, typeName, columnScalarMode, dataTypePhysicalVariation);
+    }
+  }
+
+  private void doIfTestsWithDiffColumnScalar(Random random, String typeName,
+      ColumnScalarMode columnScalarMode, DataTypePhysicalVariation 
dataTypePhysicalVariation)
+          throws Exception {
+
+    TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+
+    boolean isDecimal64 = (dataTypePhysicalVariation == 
DataTypePhysicalVariation.DECIMAL_64);
+    final int decimal64Scale =
+        (isDecimal64 ? ((DecimalTypeInfo) typeInfo).getScale() : 0);
+
+    List<String> explicitTypeNameList = new ArrayList<String>();
+    List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList = 
new ArrayList<DataTypePhysicalVariation>();
+    explicitTypeNameList.add("boolean");
+    explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
+    if (columnScalarMode != ColumnScalarMode.SCALAR_SCALAR) {
+      explicitTypeNameList.add(typeName);
+      explicitDataTypePhysicalVariationList.add(dataTypePhysicalVariation);
+      if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN) {
+        explicitTypeNameList.add(typeName);
+        explicitDataTypePhysicalVariationList.add(dataTypePhysicalVariation);
+      }
+    }
+
+    VectorRandomRowSource rowSource = new VectorRandomRowSource();
+
+    rowSource.initExplicitSchema(
+        random, explicitTypeNameList, /* maxComplexDepth */ 0, /* allowNull */ 
true,
+        explicitDataTypePhysicalVariationList);
+
+    List<String> columns = new ArrayList<String>();
+    columns.add("col0");    // The boolean predicate.
+
+    ExprNodeColumnDesc col1Expr = new  ExprNodeColumnDesc(Boolean.class, 
"col0", "table", false);
+    int columnNum = 1;
+    ExprNodeDesc col2Expr;
+    if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN ||
+        columnScalarMode == ColumnScalarMode.COLUMN_SCALAR) {
+      String columnName = "col" + (columnNum++);
+      col2Expr = new ExprNodeColumnDesc(typeInfo, columnName, "table", false);
+      columns.add(columnName);
+    } else {
+      Object scalar1Object =
+          VectorRandomRowSource.randomPrimitiveObject(
+              random, (PrimitiveTypeInfo) typeInfo);
+      col2Expr = new ExprNodeConstantDesc(typeInfo, scalar1Object);
+    }
+    ExprNodeDesc col3Expr;
+    if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN ||
+        columnScalarMode == ColumnScalarMode.SCALAR_COLUMN) {
+      String columnName = "col" + (columnNum++);
+      col3Expr = new ExprNodeColumnDesc(typeInfo, columnName, "table", false);
+      columns.add(columnName);
+    } else {
+      Object scalar2Object =
+          VectorRandomRowSource.randomPrimitiveObject(
+              random, (PrimitiveTypeInfo) typeInfo);
+      col3Expr = new ExprNodeConstantDesc(typeInfo, scalar2Object);
+    }
+
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+    children.add(col1Expr);
+    children.add(col2Expr);
+    children.add(col3Expr);
+
+    
//----------------------------------------------------------------------------------------------
+
+    String[] columnNames = columns.toArray(new String[0]);
+
+    String[] outputScratchTypeNames = new String[] { typeName };
+    DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
+        new DataTypePhysicalVariation[] { dataTypePhysicalVariation };
+
+    VectorizedRowBatchCtx batchContext =
+        new VectorizedRowBatchCtx(
+            columnNames,
+            rowSource.typeInfos(),
+            rowSource.dataTypePhysicalVariations(),
+            /* dataColumnNums */ null,
+            /* partitionColumnCount */ 0,
+            /* virtualColumnCount */ 0,
+            /* neededVirtualColumns */ null,
+            outputScratchTypeNames,
+            outputDataTypePhysicalVariations);
+
+    Object[][] randomRows = rowSource.randomRows(100000);
+
+    VectorRandomBatchSource batchSource =
+        VectorRandomBatchSource.createInterestingBatches(
+            random,
+            rowSource,
+            randomRows,
+            null);
+
+    final int rowCount = randomRows.length;
+    Object[][] resultObjectsArray = new Object[IfStmtTestMode.count][];
+    for (int i = 0; i < IfStmtTestMode.count; i++) {
+
+      Object[] resultObjects = new Object[rowCount];
+      resultObjectsArray[i] = resultObjects;
+
+      IfStmtTestMode ifStmtTestMode = IfStmtTestMode.values()[i];
+      switch (ifStmtTestMode) {
+      case ROW_MODE:
+        doRowIfTest(
+            typeInfo, columns, children, randomRows, 
rowSource.rowStructObjectInspector(),
+            resultObjects);
+        break;
+      case ADAPTOR_WHEN:
+      case VECTOR_EXPRESSION:
+        doVectorIfTest(
+            typeInfo,
+            columns,
+            rowSource.typeInfos(),
+            rowSource.dataTypePhysicalVariations(),
+            children,
+            ifStmtTestMode,
+            columnScalarMode,
+            batchSource,
+            batchContext,
+            resultObjects);
+        break;
+      default:
+        throw new RuntimeException("Unexpected IF statement test mode " + 
ifStmtTestMode);
+      }
+    }
+
+    for (int i = 0; i < rowCount; i++) {
+      // Row-mode is the expected value.
+      Object expectedResult = resultObjectsArray[0][i];
+
+      for (int v = 1; v < IfStmtTestMode.count; v++) {
+        Object vectorResult = resultObjectsArray[v][i];
+        if (expectedResult == null || vectorResult == null) {
+          if (expectedResult != null || vectorResult != null) {
+            Assert.fail(
+                "Row " + i + " " + IfStmtTestMode.values()[v] +
+                " " + columnScalarMode +
+                " result is NULL " + (vectorResult == null) +
+                " does not match row-mode expected result is NULL " + 
(expectedResult == null));
+          }
+        } else {
+
+          if (isDecimal64 && expectedResult instanceof LongWritable) {
+
+            HiveDecimalWritable expectedHiveDecimalWritable = new 
HiveDecimalWritable(0);
+            expectedHiveDecimalWritable.deserialize64(
+                ((LongWritable) expectedResult).get(), decimal64Scale);
+            expectedResult = expectedHiveDecimalWritable;
+          }
+
+          if (!expectedResult.equals(vectorResult)) {
+            Assert.fail(
+                "Row " + i + " " + IfStmtTestMode.values()[v] +
+                " " + columnScalarMode +
+                " result " + vectorResult.toString() +
+                " (" + vectorResult.getClass().getSimpleName() + ")" +
+                " does not match row-mode expected result " + 
expectedResult.toString() +
+                " (" + expectedResult.getClass().getSimpleName() + ")");
+          }
+        }
+      }
+    }
+  }
+
+  private void doRowIfTest(TypeInfo typeInfo, List<String> columns, 
List<ExprNodeDesc> children,
+      Object[][] randomRows, ObjectInspector rowInspector, Object[] 
resultObjects) throws Exception {
+
+    GenericUDF udf = new GenericUDFIf();
+
+    ExprNodeGenericFuncDesc exprDesc =
+        new ExprNodeGenericFuncDesc(typeInfo, udf, children);
+    HiveConf hiveConf = new HiveConf();
+    ExprNodeEvaluator evaluator =
+        ExprNodeEvaluatorFactory.get(exprDesc, hiveConf);
+    evaluator.initialize(rowInspector);
+
+    final int rowCount = randomRows.length;
+    for (int i = 0; i < rowCount; i++) {
+      Object[] row = randomRows[i];
+      Object result = evaluator.evaluate(row);
+      resultObjects[i] = result;
+    }
+  }
+
+  private void extractResultObjects(VectorizedRowBatch batch, int rowIndex,
+      VectorExtractRow resultVectorExtractRow, Object[] scrqtchRow, Object[] 
resultObjects) {
+    // UNDONE: selectedInUse
+    for (int i = 0; i < batch.size; i++) {
+      resultVectorExtractRow.extractRow(batch, i, scrqtchRow);
+
+      // UNDONE: Need to copy the object.
+      resultObjects[rowIndex++] = scrqtchRow[0];
+    }
+  }
+
+  private void doVectorIfTest(TypeInfo typeInfo,
+      List<String> columns,
+      TypeInfo[] typeInfos, DataTypePhysicalVariation[] 
dataTypePhysicalVariations,
+      List<ExprNodeDesc> children,
+      IfStmtTestMode ifStmtTestMode, ColumnScalarMode columnScalarMode,
+      VectorRandomBatchSource batchSource, VectorizedRowBatchCtx batchContext,
+      Object[] resultObjects)
+          throws Exception {
+
+    GenericUDF udf;
+    switch (ifStmtTestMode) {
+    case VECTOR_EXPRESSION:
+      udf = new GenericUDFIf();
+      break;
+    case ADAPTOR_WHEN:
+      udf = new GenericUDFWhen();
+      break;
+    default:
+      throw new RuntimeException("Unexpected IF statement test mode " + 
ifStmtTestMode);
+    }
+
+    ExprNodeGenericFuncDesc exprDesc =
+        new ExprNodeGenericFuncDesc(typeInfo, udf, children);
+
+    String ifExprMode = (ifStmtTestMode != IfStmtTestMode.VECTOR_EXPRESSION ? 
"adaptor" : "good");
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_VECTORIZED_IF_EXPR_MODE, 
ifExprMode);
+
+    VectorizationContext vectorizationContext =
+        new VectorizationContext(
+            "name",
+            columns,
+            Arrays.asList(typeInfos),
+            Arrays.asList(dataTypePhysicalVariations),
+            hiveConf);
+    VectorExpression vectorExpression = 
vectorizationContext.getVectorExpression(exprDesc);
+
+    VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
+
+    VectorExtractRow resultVectorExtractRow = new VectorExtractRow();
+    resultVectorExtractRow.init(new TypeInfo[] { typeInfo }, new int[] { 
columns.size() });
+    Object[] scrqtchRow = new Object[1];
+
+    /*
+    System.out.println(
+        "*DEBUG* typeInfo " + typeInfo.toString() +
+        " ifStmtTestMode " + ifStmtTestMode +
+        " columnScalarMode " + columnScalarMode +
+        " vectorExpression " + vectorExpression.getClass().getSimpleName());
+    */
+
+    batchSource.resetBatchIteration();
+    int rowIndex = 0;
+    while (true) {
+      if (!batchSource.fillNextBatch(batch)) {
+        break;
+      }
+      vectorExpression.evaluate(batch);
+      extractResultObjects(batch, rowIndex, resultVectorExtractRow, 
scrqtchRow, resultObjects);
+      rowIndex += batch.size;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q 
b/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
index afbc18a..859289c 100644
--- a/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
+++ b/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
@@ -1,22 +1,40 @@
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.fetch.task.conversion=none;
+SET hive.vectorized.execution.enabled = false;
+
 -- Test timestamp functions in vectorized mode to verify they run correctly 
end-to-end.
 -- Turning on vectorization has been temporarily moved after filling the test 
table
 -- due to bug HIVE-8197.
 
+-- SORT_QUERY_RESULTS
 
-CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) 
STORED AS ORC;
+CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 timestamp, 
stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC;
 
 INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40;
-
-SET hive.vectorized.execution.enabled = true;
+INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', 
'1978-08-05 14:41:05.501', '1999-10-03 16:59:10.396903939');
+INSERT INTO TABLE alltypesorc_string values (false, null, '2013-04-10 
00:43:46.8547315', null);
+INSERT INTO TABLE alltypesorc_string values (false, '2021-09-24 03:18:32.4', 
null, null);
+INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 
06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183');
+INSERT INTO TABLE alltypesorc_string values (null, null, '4966-12-04 
09:30:55.202', null);
+INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 
06:00:24.81200852', null, null);
+INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 09:30:11.0', 
'8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778');
+INSERT INTO TABLE alltypesorc_string values (true, null, '1883-04-17 
04:14:34.64776', '2024-11-11 16:42:41.101');
+INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 
08:15:18.941718273', null, null);
+
+INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 
02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989');
 
 CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC;
 
@@ -24,6 +42,8 @@ INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM 
alltypesorc LIMIT 1;
 INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' FROM 
alltypesorc LIMIT 1;
 INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' FROM 
alltypesorc LIMIT 1;
 
+SET hive.vectorized.execution.enabled = true;
+
 EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(ctimestamp1) AS c1,
   year(ctimestamp1),
@@ -33,7 +53,15 @@ EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1;
 
@@ -46,7 +74,15 @@ SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e2d4f347/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out 
b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
index c9dd434..79ba4c6 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
@@ -1,15 +1,19 @@
-PREHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, 
stimestamp1 string) STORED AS ORC
+PREHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, ctimestamp1 
timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@alltypesorc_string
-POSTHOOK: query: CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, 
stimestamp1 string) STORED AS ORC
+POSTHOOK: query: CREATE TABLE alltypesorc_string(cboolean1 boolean, 
ctimestamp1 timestamp, stimestamp1 string,
+    ctimestamp2 timestamp) STORED AS ORC
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@alltypesorc_string
 PREHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
@@ -18,16 +22,146 @@ PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@alltypesorc_string
 POSTHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string
 SELECT
+  cboolean1,
   to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS toutc,
-  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) as cst,
+  ctimestamp2
 FROM alltypesorc
 ORDER BY toutc, cst
 LIMIT 40
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc
 POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SIMPLE 
[(alltypesorc)alltypesorc.FieldSchema(name:cboolean1, type:boolean, 
comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION 
[(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, 
comment:null), ]
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SIMPLE 
[(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp2, type:timestamp, 
comment:null), ]
 POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION 
[(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, 
comment:null), ]
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, 
'2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 
16:59:10.396903939')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, 
'2021-09-24 03:18:32.4', '1978-08-05 14:41:05.501', '1999-10-03 
16:59:10.396903939')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, 
'2013-04-10 00:43:46.8547315', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, null, 
'2013-04-10 00:43:46.8547315', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, 
'2021-09-24 03:18:32.4', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (false, 
'2021-09-24 03:18:32.4', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 
06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 13:36:50.183')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, 
'7160-12-02 06:00:24.81200852', '0004-09-22 18:26:29.519542222', '1966-08-16 
13:36:50.183')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, 
'4966-12-04 09:30:55.202', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, null, 
'4966-12-04 09:30:55.202', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, '7160-12-02 
06:00:24.81200852', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (null, 
'7160-12-02 06:00:24.81200852', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '1985-07-20 
09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 16:31:57.778')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, 
'1985-07-20 09:30:11.0', '8521-01-16 20:42:05.668832', '1319-02-02 
16:31:57.778')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, 
'1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, null, 
'1883-04-17 04:14:34.64776', '2024-11-11 16:42:41.101')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, '0528-10-27 
08:15:18.941718273', null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values (true, 
'0528-10-27 08:15:18.941718273', null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 EXPRESSION []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 EXPRESSION []
+PREHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 
02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@alltypesorc_string
+POSTHOOK: query: INSERT INTO TABLE alltypesorc_string values
+     (false, '2021-09-24 03:18:32.4', '1985-11-18 16:37:54.0', '2010-04-08 
02:43:35.861742727'),
+     (true, null, '1985-11-18 16:37:54.0', null),
+     (null, '2021-09-24 03:18:32.4', null, '1974-10-04 17:21:03.989')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@alltypesorc_string
+POSTHOOK: Lineage: alltypesorc_string.cboolean1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp1 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.ctimestamp2 SCRIPT []
+POSTHOOK: Lineage: alltypesorc_string.stimestamp1 SCRIPT []
 PREHOOK: query: CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS 
ORC
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -72,7 +206,15 @@ PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -85,7 +227,15 @@ POSTHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
@@ -109,18 +259,18 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 4356 Basic stats: 
COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
-                    expressions: to_unix_timestamp(ctimestamp1) (type: 
bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), 
day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), 
weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), 
minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6, _col7, _col8
+                    expressions: to_unix_timestamp(ctimestamp1) (type: 
bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), 
day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), 
weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), 
minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int), cboolean1 
(type: boolean), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), 
if(cboolean1, ctimestamp1, TIMESTAMP'1319-02-02 16:31:57.778') (type: 
timestamp), if(cboolean1, TIMESTAMP'2000-12-18 08:42:30.0005', ctimestamp1) 
(type: timestamp), if(cboolean1, ctimestamp1, ctimestamp2) (type: timestamp), 
if(cboolean1, ctimestamp1, null) (type: timestamp), if(cboolean1, null, 
ctimestamp2) (type: timestamp)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, 
_col15, _col16
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 
11]
-                        selectExpressions: VectorUDFUnixTimeStampTimestamp(col 
0:timestamp) -> 3:bigint, VectorUDFYearTimestamp(col 0:timestamp, field YEAR) 
-> 4:int, VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 5:int, 
VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 6:int, 
VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 7:int, 
VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 8:int, 
VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 9:int, 
VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 10:int, 
VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 11:int
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 
13, 0, 1, 3, 14, 15, 16, 17, 18]
+                        selectExpressions: VectorUDFUnixTimeStampTimestamp(col 
1:timestamp) -> 5:bigint, VectorUDFYearTimestamp(col 1:timestamp, field YEAR) 
-> 6:int, VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 7:int, 
VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 8:int, 
VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 9:int, 
VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 10:int, 
VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 11:int, 
VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 12:int, 
VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 13:int, 
IfExprTimestampColumnScalar(col 0:boolean, col 1:timestamp, val 1319-02-02 
16:31:57.778) -> 14:timestamp, IfExprTimestampScalarColumn(col 0:boolean, val 
2000-12-18 08:42:30.0005, col 1:timestamp) -> 15:timestamp, 
IfExprTimestampColumnColumn(col 0:boolean, col 1:timestampcol 3:timestamp) -> 
16:timestamp, IfExprColumnNull(col 0:boole
 an, col 1:timestamp, null)(children: col 0:boolean, col 1:timestamp) -> 
17:timestamp, IfExprNullColumn(col 0:boolean, null, col 3)(children: col 
0:boolean, col 3:timestamp) -> 18:timestamp
+                    Statistics: Num rows: 52 Data size: 16836 Basic stats: 
COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -128,8 +278,8 @@ STAGE PLANS:
                           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: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
-                      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)
+                      Statistics: Num rows: 52 Data size: 16836 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      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), _col9 (type: boolean), _col10 (type: 
timestamp), _col11 (type: timestamp), _col12 (type: timestamp), _col13 (type: 
timestamp), _col14 (type: timestamp), _col15 (type: timestamp), _col16 (type: 
timestamp)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -151,19 +301,19 @@ STAGE PLANS:
                 vectorized: true
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 
(type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 
(type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 
(type: int), VALUE._col7 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6, _col7, _col8
+                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 
(type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 
(type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 
(type: int), VALUE._col7 (type: int), VALUE._col8 (type: boolean), VALUE._col9 
(type: timestamp), VALUE._col10 (type: timestamp), VALUE._col11 (type: 
timestamp), VALUE._col12 (type: timestamp), VALUE._col13 (type: timestamp), 
VALUE._col14 (type: timestamp), VALUE._col15 (type: timestamp)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, 
_col16
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE 
Column stats: COMPLETE
+                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 
10, 11, 12, 13, 14, 15, 16]
+                Statistics: Num rows: 52 Data size: 16836 Basic stats: 
COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 16836 Basic stats: 
COMPLETE Column stats: COMPLETE
                   table:
                       input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -184,7 +334,15 @@ PREHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 PREHOOK: type: QUERY
@@ -199,52 +357,72 @@ POSTHOOK: query: SELECT
   weekofyear(ctimestamp1),
   hour(ctimestamp1),
   minute(ctimestamp1),
-  second(ctimestamp1)
+  second(ctimestamp1),
+  cboolean1,
+  ctimestamp1,
+  ctimestamp2,
+  if (cboolean1, ctimestamp1, timestamp '1319-02-02 16:31:57.778'),
+  if (cboolean1, timestamp '2000-12-18 08:42:30.0005', ctimestamp1),
+  if (cboolean1, ctimestamp1, ctimestamp2),
+  if (cboolean1, ctimestamp1, null),
+  if (cboolean1, null, ctimestamp2)
 FROM alltypesorc_string
 ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+-45479000681   528     10      27      27      43      8       15      18      
true    0528-10-27 08:15:18.941718273   NULL    0528-10-27 08:15:18.941718273   
2000-12-18 08:42:30.0005        0528-10-27 08:15:18.941718273   0528-10-27 
08:15:18.941718273   NULL
+1632478712     2021    9       24      24      38      3       18      32      
NULL    2021-09-24 03:18:32.4   1974-10-04 17:21:03.989 1319-02-02 16:31:57.778 
2021-09-24 03:18:32.4   1974-10-04 17:21:03.989 NULL    1974-10-04 17:21:03.989
+1632478712     2021    9       24      24      38      3       18      32      
false   2021-09-24 03:18:32.4   1999-10-03 16:59:10.396903939   1319-02-02 
16:31:57.778 2021-09-24 03:18:32.4   1999-10-03 16:59:10.396903939   NULL    
1999-10-03 16:59:10.396903939
+1632478712     2021    9       24      24      38      3       18      32      
false   2021-09-24 03:18:32.4   2010-04-08 02:43:35.861742727   1319-02-02 
16:31:57.778 2021-09-24 03:18:32.4   2010-04-08 02:43:35.861742727   NULL    
2010-04-08 02:43:35.861742727
+1632478712     2021    9       24      24      38      3       18      32      
false   2021-09-24 03:18:32.4   NULL    1319-02-02 16:31:57.778 2021-09-24 
03:18:32.4   NULL    NULL    NULL
+163809612024   7160    12      2       2       48      6       0       24      
NULL    7160-12-02 06:00:24.81200852    1966-08-16 13:36:50.183 1319-02-02 
16:31:57.778 7160-12-02 06:00:24.81200852    1966-08-16 13:36:50.183 NULL    
1966-08-16 13:36:50.183
+163809612024   7160    12      2       2       48      6       0       24      
NULL    7160-12-02 06:00:24.81200852    NULL    1319-02-02 16:31:57.778 
7160-12-02 06:00:24.81200852    NULL    NULL    NULL
+490725011      1985    7       20      20      29      9       30      11      
true    1985-07-20 09:30:11     1319-02-02 16:31:57.778 1985-07-20 09:30:11     
2000-12-18 08:42:30.0005        1985-07-20 09:30:11     1985-07-20 09:30:11     
NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    
NULL    NULL    1319-02-02 16:31:57.778 NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 15:59:44.028 1319-02-02 16:31:57.778 NULL    1969-12-31 
15:59:44.028 NULL    1969-12-31 15:59:44.028
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 15:59:44.809 1319-02-02 16:31:57.778 NULL    1969-12-31 
15:59:44.809 NULL    1969-12-31 15:59:44.809
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 15:59:50.531 1319-02-02 16:31:57.778 NULL    1969-12-31 
15:59:50.531 NULL    1969-12-31 15:59:50.531
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 15:59:51.009 1319-02-02 16:31:57.778 NULL    1969-12-31 
15:59:51.009 NULL    1969-12-31 15:59:51.009
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 15:59:53.761 1319-02-02 16:31:57.778 NULL    1969-12-31 
15:59:53.761 NULL    1969-12-31 15:59:53.761
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:00.905 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:00.905 NULL    1969-12-31 16:00:00.905
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:03.586 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:03.586 NULL    1969-12-31 16:00:03.586
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:05.227 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:05.227 NULL    1969-12-31 16:00:05.227
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:05.535 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:05.535 NULL    1969-12-31 16:00:05.535
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:07.02  1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:07.02  NULL    1969-12-31 16:00:07.02
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:07.365 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:07.365 NULL    1969-12-31 16:00:07.365
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:07.517 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:07.517 NULL    1969-12-31 16:00:07.517
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:07.767 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:07.767 NULL    1969-12-31 16:00:07.767
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:08.602 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:08.602 NULL    1969-12-31 16:00:08.602
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:09.938 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:09.938 NULL    1969-12-31 16:00:09.938
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:14.214 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:14.214 NULL    1969-12-31 16:00:14.214
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    1969-12-31 16:00:14.783 1319-02-02 16:31:57.778 NULL    1969-12-31 
16:00:14.783 NULL    1969-12-31 16:00:14.783
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    NULL    1319-02-02 16:31:57.778 NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    false   
NULL    NULL    1319-02-02 16:31:57.778 NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:43.773 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:44.262 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:44.568 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:47.351 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:47.446 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:48.023 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:48.629 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:49.177 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:49.208 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:50.789 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:51.245 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:52.372 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 15:59:55.249 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:00.661 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:00.784 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:01.836 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:09.313 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:09.538 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:09.986 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:11.031 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:11.465 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    1969-12-31 16:00:13.589 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    2024-11-11 16:42:41.101 NULL    2000-12-18 08:42:30.0005        NULL    
NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    true    
NULL    NULL    NULL    2000-12-18 08:42:30.0005        NULL    NULL    NULL
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -291,7 +469,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 84 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1017 Basic stats: 
COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -300,9 +478,9 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 9, 10, 
11]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 
1:string) -> 3:bigint, VectorUDFYearString(col 1:string, fieldStart 0, 
fieldLength 4) -> 4:int, VectorUDFMonthString(col 1:string, fieldStart 5, 
fieldLength 2) -> 5:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, 
fieldLength 2) -> 6:int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, 
fieldLength 2) -> 7:int, VectorUDFWeekOfYearString(col 1:string) -> 8:int, 
VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 9:int, 
VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 10:int, 
VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 11:int
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 
13]
+                        selectExpressions: VectorUDFUnixTimeStampString(col 
2:string) -> 5:bigint, VectorUDFYearString(col 2:string, fieldStart 0, 
fieldLength 4) -> 6:int, VectorUDFMonthString(col 2:string, fieldStart 5, 
fieldLength 2) -> 7:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, 
fieldLength 2) -> 8:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, 
fieldLength 2) -> 9:int, VectorUDFWeekOfYearString(col 2:string) -> 10:int, 
VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 11:int, 
VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 12:int, 
VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 13:int
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
                       sort order: +
@@ -310,7 +488,7 @@ STAGE PLANS:
                           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: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                       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
             LLAP IO: all inputs
@@ -339,13 +517,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 1600 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE 
Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                   table:
                       input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -387,6 +565,18 @@ ORDER BY c1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
+-2736243926    1883    4       17      17      16      4       14      34
+-62018170411   4       9       22      22      39      18      26      29
+1365579826     2013    4       10      10      15      0       43      46
+206731024925   8521    1       16      16      3       20      42      5
+271201265      1978    8       5       5       31      14      41      5
+501208674      1985    11      18      18      47      16      37      54
+501208674      1985    11      18      18      47      16      37      54
+94573848655    4966    12      4       4       49      9       30      55
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
 NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
 NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
 NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
@@ -473,7 +663,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1684 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 3097 Basic stats: 
COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -482,9 +672,9 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [5, 6, 7, 8, 9, 10, 11, 12, 
13]
-                        selectExpressions: LongColEqualLongColumn(col 
3:bigint, col 4:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 
0:timestamp) -> 3:bigint, VectorUDFUnixTimeStampString(col 1:string) -> 
4:bigint) -> 5:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFYearTimestamp(col 0:timestamp, field YEAR) -> 3:int, 
VectorUDFYearString(col 1:string, fieldStart 0, fieldLength 4) -> 4:int) -> 
6:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFMonthTimestamp(col 0:timestamp, field MONTH) -> 3:int, 
VectorUDFMonthString(col 1:string, fieldStart 5, fieldLength 2) -> 4:int) -> 
7:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:int, 
VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 4:int) 
-> 8:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFDayOfMonthTimestamp(col 0:timestamp, field DAY_OF_MONTH) -> 3:
 int, VectorUDFDayOfMonthString(col 1:string, fieldStart 8, fieldLength 2) -> 
4:int) -> 9:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFWeekOfYearTimestamp(col 0:timestamp, field WEEK_OF_YEAR) -> 3:int, 
VectorUDFWeekOfYearString(col 1:string) -> 4:int) -> 10:boolean, 
LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFHourTimestamp(col 0:timestamp, field HOUR_OF_DAY) -> 3:int, 
VectorUDFHourString(col 1:string, fieldStart 11, fieldLength 2) -> 4:int) -> 
11:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFMinuteTimestamp(col 0:timestamp, field MINUTE) -> 3:int, 
VectorUDFMinuteString(col 1:string, fieldStart 14, fieldLength 2) -> 4:int) -> 
12:boolean, LongColEqualLongColumn(col 3:int, col 4:int)(children: 
VectorUDFSecondTimestamp(col 0:timestamp, field SECOND) -> 3:int, 
VectorUDFSecondString(col 1:string, fieldStart 17, fieldLength 2) -> 4:int) -> 
13:boolean
-                    Statistics: Num rows: 40 Data size: 1440 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [7, 8, 9, 10, 11, 12, 13, 
14, 15]
+                        selectExpressions: LongColEqualLongColumn(col 
5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 
1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 
6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, 
VectorUDFYearString(col 2:string, fieldStart 0, fieldLength 4) -> 6:int) -> 
8:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, 
VectorUDFMonthString(col 2:string, fieldStart 5, fieldLength 2) -> 6:int) -> 
9:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, 
VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int) 
-> 10:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5
 :int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 
6:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, 
VectorUDFWeekOfYearString(col 2:string) -> 6:int) -> 12:boolean, 
LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, 
VectorUDFHourString(col 2:string, fieldStart 11, fieldLength 2) -> 6:int) -> 
13:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, 
VectorUDFMinuteString(col 2:string, fieldStart 14, fieldLength 2) -> 6:int) -> 
14:boolean, LongColEqualLongColumn(col 5:int, col 6:int)(children: 
VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, 
VectorUDFSecondString(col 2:string, fieldStart 17, fieldLength 2) -> 6:int) -> 
15:boolean
+                    Statistics: Num rows: 52 Data size: 1872 Basic stats: 
COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: boolean)
                       sort order: +
@@ -492,7 +682,7 @@ STAGE PLANS:
                           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: 40 Data size: 1440 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 52 Data size: 1872 Basic stats: 
COMPLETE Column stats: COMPLETE
                       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
             LLAP IO: all inputs
@@ -521,13 +711,13 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
-                Statistics: Num rows: 40 Data size: 1440 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 1872 Basic stats: COMPLETE 
Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-                  Statistics: Num rows: 40 Data size: 1440 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 1872 Basic stats: 
COMPLETE Column stats: COMPLETE
                   table:
                       input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -609,6 +799,18 @@ NULL       NULL    NULL    NULL    NULL    NULL    NULL    
NULL    NULL
 NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
 NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
 NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL
+false  false   false   false   false   false   false   false   false
+false  false   false   false   false   false   false   false   false
+false  false   false   false   false   false   false   false   false
+false  false   false   false   false   false   false   false   false
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   to_unix_timestamp(stimestamp1) AS c1,
   year(stimestamp1),
@@ -788,7 +990,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -797,12 +999,12 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [1]
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: min(ctimestamp1), max(ctimestamp1), 
count(ctimestamp1), count()
                       Group By Vectorization:
-                          aggregators: VectorUDAFMinTimestamp(col 0:timestamp) 
-> timestamp, VectorUDAFMaxTimestamp(col 0:timestamp) -> timestamp, 
VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
+                          aggregators: VectorUDAFMinTimestamp(col 1:timestamp) 
-> timestamp, VectorUDAFMaxTimestamp(col 1:timestamp) -> timestamp, 
VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFCountStar(*) -> bigint
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -886,7 +1088,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL   NULL    0       40
+0528-10-27 08:15:18.941718273  7160-12-02 06:00:24.81200852    8       52
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(sum(ctimestamp1), 3)
 FROM alltypesorc_string
@@ -915,7 +1117,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -924,12 +1126,12 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [1]
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: sum(ctimestamp1)
                       Group By Vectorization:
-                          aggregators: VectorUDAFSumTimestamp(col 0:timestamp) 
-> double
+                          aggregators: VectorUDAFSumTimestamp(col 1:timestamp) 
-> double
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -1016,7 +1218,7 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL
+2.89160863229166E11
 PREHOOK: query: EXPLAIN VECTORIZATION EXPRESSION  SELECT
   round(avg(ctimestamp1), 0),
   variance(ctimestamp1) between 8.97077295279421E19 and 8.97077295279422E19,
@@ -1059,7 +1261,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: alltypesorc_string
-                  Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
                   Select Operator
@@ -1068,13 +1270,13 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 3, 6]
-                        selectExpressions: CastTimestampToDouble(col 
0:timestamp) -> 3:double, DoubleColMultiplyDoubleColumn(col 4:double, col 
5:double)(children: CastTimestampToDouble(col 0:timestamp) -> 4:double, 
CastTimestampToDouble(col 0:timestamp) -> 5:double) -> 6:double
-                    Statistics: Num rows: 40 Data size: 1600 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        projectedOutputColumnNums: [1, 5, 8]
+                        selectExpressions: CastTimestampToDouble(col 
1:timestamp) -> 5:double, DoubleColMultiplyDoubleColumn(col 6:double, col 
7:double)(children: CastTimestampToDouble(col 1:timestamp) -> 6:double, 
CastTimestampToDouble(col 1:timestamp) -> 7:double) -> 8:double
+                    Statistics: Num rows: 52 Data size: 2080 Basic stats: 
COMPLETE Column stats: COMPLETE
                     Group By Operator
                       aggregations: sum(_col0), count(_col0), sum(_col2), 
sum(_col1)
                       Group By Vectorization:
-                          aggregators: VectorUDAFSumTimestamp(col 0:timestamp) 
-> double, VectorUDAFCount(col 0:timestamp) -> bigint, VectorUDAFSumDouble(col 
6:double) -> double, VectorUDAFSumDouble(col 3:double) -> double
+                          aggregators: VectorUDAFSumTimestamp(col 1:timestamp) 
-> double, VectorUDAFCount(col 1:timestamp) -> bigint, VectorUDAFSumDouble(col 
8:double) -> double, VectorUDAFSumDouble(col 5:double) -> double
                           className: VectorGroupByOperator
                           groupByMode: HASH
                           native: false
@@ -1175,4 +1377,4 @@ FROM alltypesorc_string
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc_string
 #### A masked pattern was here ####
-NULL   NULL    NULL    NULL    NULL    NULL    NULL    NULL
+3.6145107904E10        false   false   false   7.5245155692476E10      
7.5245155692476E10      7.5245155692476E10      8.0440455033059E10

Reply via email to