You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ku...@apache.org on 2019/07/10 09:23:35 UTC

[hive] branch master updated: HIVE-21868: Vectorize CAST...FORMAT (Karen Coppage via Marta Kuczora)

This is an automated email from the ASF dual-hosted git repository.

kuczoram pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new de6d005  HIVE-21868: Vectorize CAST...FORMAT (Karen Coppage via Marta Kuczora)
de6d005 is described below

commit de6d005501f89d78b67d8665d1104e7dd9d64359
Author: Karen Coppage <kc...@gmail.com>
AuthorDate: Wed Jul 10 11:22:37 2019 +0200

    HIVE-21868: Vectorize CAST...FORMAT (Karen Coppage via Marta Kuczora)
---
 .../format/datetime/HiveSqlDateTimeFormatter.java  |   5 +-
 .../hive/ql/exec/vector/VectorizationContext.java  |  61 +++++++++
 ...ToString.java => CastDateToCharWithFormat.java} |  43 +++---
 .../exec/vector/expressions/CastDateToString.java  |  17 +++
 ...String.java => CastDateToStringWithFormat.java} |  41 +++---
 ...tring.java => CastDateToVarCharWithFormat.java} |  43 +++---
 .../exec/vector/expressions/CastStringToDate.java  |   5 +-
 .../expressions/CastStringToDateWithFormat.java    |  63 +++++++++
 .../vector/expressions/CastStringToTimestamp.java  |  12 +-
 .../CastStringToTimestampWithFormat.java           |  64 +++++++++
 .../expressions/CastTimestampToCharWithFormat.java |  58 ++++++++
 .../vector/expressions/CastTimestampToString.java  |  17 +++
 .../CastTimestampToStringWithFormat.java           |  57 ++++++++
 .../CastTimestampToVarCharWithFormat.java          |  58 ++++++++
 .../hive/ql/udf/generic/GenericUDFCastFormat.java  |  41 ++++--
 .../expressions/TestVectorMathFunctions.java       | 126 +++++++++++++++++-
 .../vector/expressions/TestVectorTypeCasts.java    |  51 +++++++
 .../expressions/TestVectorTypeCastsWithFormat.java | 147 +++++++++++++++++++++
 .../clientnegative/udf_cast_format_bad_pattern.q   |   1 +
 .../cast_datetime_with_sql_2016_format.q           |  21 +++
 .../udf_cast_format_bad_pattern.q.out              |   1 +
 .../cast_datetime_with_sql_2016_format.q.out       | 138 +++++++++++++++++++
 22 files changed, 985 insertions(+), 85 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/common/format/datetime/HiveSqlDateTimeFormatter.java b/common/src/java/org/apache/hadoop/hive/common/format/datetime/HiveSqlDateTimeFormatter.java
index 4e024a3..998e5a2 100644
--- a/common/src/java/org/apache/hadoop/hive/common/format/datetime/HiveSqlDateTimeFormatter.java
+++ b/common/src/java/org/apache/hadoop/hive/common/format/datetime/HiveSqlDateTimeFormatter.java
@@ -24,6 +24,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.Timestamp;
 
+import java.io.Serializable;
 import java.time.DateTimeException;
 import java.time.Duration;
 import java.time.Instant;
@@ -211,7 +212,7 @@ import java.util.TimeZone;
  * See T.
  */
 
-public class HiveSqlDateTimeFormatter {
+public class HiveSqlDateTimeFormatter implements Serializable {
 
   private static final int LONGEST_TOKEN_LENGTH = 5;
   private static final int LONGEST_ACCEPTED_PATTERN = 100; // for sanity's sake
@@ -274,7 +275,7 @@ public class HiveSqlDateTimeFormatter {
   /**
    * Token representation.
    */
-  public static class Token {
+  public static class Token implements Serializable {
     TokenType type;
     TemporalField temporalField; // for type TEMPORAL e.g. ChronoField.YEAR
     TemporalUnit temporalUnit; // for type TIMEZONE e.g. ChronoUnit.HOURS
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index fa9d1e9..51d92a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -42,8 +42,11 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.CastBooleanToStringViaL
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastBooleanToVarCharViaLongToVarChar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastCharToBinary;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToChar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToCharWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToString;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToStringWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToVarChar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToVarCharWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToChar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToString;
@@ -65,12 +68,17 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.CastMillisecondsLongToT
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringGroupToChar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringGroupToVarChar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToBoolean;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToDateWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToTimestampWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToChar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToCharWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToString;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToStringWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToVarChar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToVarCharWithFormat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ConstantVectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ConvertDecimal64ToDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.Decimal64ColumnInList;
@@ -155,6 +163,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.TimestampColumnBetw
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.TimestampColumnNotBetween;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.VarCharColumnBetween;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.VarCharColumnNotBetween;
+import org.apache.hadoop.hive.serde.serdeConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
@@ -2290,6 +2299,8 @@ import com.google.common.annotations.VisibleForTesting;
       ve = new BucketNumExpression(outCol);
       ve.setInputTypeInfos(returnType);
       ve.setOutputTypeInfo(returnType);
+    } else if (udf instanceof GenericUDFCastFormat) {
+      ve = getCastWithFormat(udf, childExpr, returnType);
     }
     if (ve != null) {
       return ve;
@@ -3300,6 +3311,56 @@ import com.google.common.annotations.VisibleForTesting;
     return null;
   }
 
+  private VectorExpression getCastWithFormat(
+      GenericUDF udf, List<ExprNodeDesc> childExpr, TypeInfo returnType) throws HiveException {
+    String inputType = childExpr.get(1).getTypeString();
+    childExpr.remove(0); // index 0 not needed since we know returnType
+
+    Class<?> veClass = getCastFormatVectorExpressionClass(childExpr, returnType, inputType);
+    return createVectorExpression(
+        veClass, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+  }
+
+  private Class<?> getCastFormatVectorExpressionClass(List<ExprNodeDesc> childExpr,
+      TypeInfo returnType, String inputType) throws HiveException {
+    switch (inputType) {
+    case serdeConstants.TIMESTAMP_TYPE_NAME:
+      if (returnType.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
+        return CastTimestampToStringWithFormat.class;
+      }
+      if (returnType.getTypeName().startsWith(serdeConstants.VARCHAR_TYPE_NAME)) {
+        return CastTimestampToVarCharWithFormat.class;
+      }
+      if (returnType.getTypeName().startsWith(serdeConstants.CHAR_TYPE_NAME)) {
+        return CastTimestampToCharWithFormat.class;
+      }
+    case serdeConstants.DATE_TYPE_NAME:
+      if (returnType.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
+        return CastDateToStringWithFormat.class;
+      }
+      if (returnType.getTypeName().startsWith(serdeConstants.VARCHAR_TYPE_NAME)) {
+        return CastDateToVarCharWithFormat.class;
+      }
+      if (returnType.getTypeName().startsWith(serdeConstants.CHAR_TYPE_NAME)) {
+        return CastDateToCharWithFormat.class;
+      }
+    default: //keep going
+    }
+    if (inputType.equals(serdeConstants.STRING_TYPE_NAME)
+        || inputType.startsWith(serdeConstants.CHAR_TYPE_NAME)
+        || inputType.startsWith(serdeConstants.VARCHAR_TYPE_NAME)) {
+      switch (returnType.getTypeName()) {
+      case serdeConstants.TIMESTAMP_TYPE_NAME:
+        return CastStringToTimestampWithFormat.class;
+      case serdeConstants.DATE_TYPE_NAME:
+        return CastStringToDateWithFormat.class;
+      default: //keep going
+      }
+    }
+    throw new HiveException(
+        "Expression cast " + inputType + " to " + returnType + " format not" + " vectorizable");
+  }
+
   private VectorExpression tryDecimal64Between(VectorExpressionDescriptor.Mode mode, boolean isNot,
       ExprNodeDesc colExpr, List<ExprNodeDesc> childrenAfterNot, TypeInfo returnTypeInfo)
           throws HiveException {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToCharWithFormat.java
similarity index 50%
copy from ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
copy to ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToCharWithFormat.java
index dfa9f8a..17afbc3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToCharWithFormat.java
@@ -18,39 +18,40 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 
-import java.sql.Date;
-import java.text.SimpleDateFormat;
-import java.util.TimeZone;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<DATE> TO CHAR(<LENGTH>) WITH FORMAT <STRING>).
+ */
+public class CastDateToCharWithFormat extends CastDateToChar {
 
-public class CastDateToString extends LongToStringUnaryUDF {
   private static final long serialVersionUID = 1L;
-  protected transient Date dt = new Date(0);
-  private transient SimpleDateFormat formatter;
+  private HiveSqlDateTimeFormatter sqlFormatter;
 
-  public CastDateToString() {
+  public CastDateToCharWithFormat() {
     super();
-    formatter = new SimpleDateFormat("yyyy-MM-dd");
-    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
   }
 
-  public CastDateToString(int inputColumn, int outputColumnNum) {
+  public CastDateToCharWithFormat(int inputColumn, byte[] patternBytes, int len,
+      int outputColumnNum) {
     super(inputColumn, outputColumnNum);
-    formatter = new SimpleDateFormat("yyyy-MM-dd");
-    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+    if (patternBytes == null) {
+      throw new IllegalStateException(
+          "Tried to cast (<date> to char with format <pattern>)," + " but <pattern> not found");
+    }
+    sqlFormatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), false);
   }
 
-  // The assign method will be overridden for CHAR and VARCHAR.
-  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
-    outV.setVal(i, bytes, 0, length);
+  @Override protected void func(BytesColumnVector outV, long[] vector, int i) {
+    super.sqlFormat(outV, vector, i, sqlFormatter);
   }
 
-  @Override
-  protected void func(BytesColumnVector outV, long[] vector, int i) {
-    dt.setTime(DateWritableV2.daysToMillis((int) vector[i]));
-    byte[] temp = formatter.format(dt).getBytes();
-    assign(outV, i, temp, temp.length);
+  @Override public String vectorExpressionParameters() {
+    return super.vectorExpressionParameters() + ", format pattern: " + sqlFormatter.getPattern();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
index dfa9f8a..8a07232 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 
@@ -53,4 +54,20 @@ public class CastDateToString extends LongToStringUnaryUDF {
     byte[] temp = formatter.format(dt).getBytes();
     assign(outV, i, temp, temp.length);
   }
+
+  /**
+   * CastDateToString, CastDateToChar, CastDateToVarchar use this.
+   */
+  void sqlFormat(BytesColumnVector outV, long[] vector, int i,
+      HiveSqlDateTimeFormatter sqlFormatter) {
+    String formattedDate =
+        sqlFormatter.format(org.apache.hadoop.hive.common.type.Date.ofEpochDay((int) vector[i]));
+    if (formattedDate == null) {
+      outV.isNull[i] = true;
+      outV.noNulls = false;
+      return;
+    }
+    byte[] temp = formattedDate.getBytes();
+    assign(outV, i, temp, temp.length);
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToStringWithFormat.java
similarity index 54%
copy from ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
copy to ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToStringWithFormat.java
index dfa9f8a..acfa2ba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToStringWithFormat.java
@@ -18,39 +18,40 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 
-import java.sql.Date;
-import java.text.SimpleDateFormat;
-import java.util.TimeZone;
+import java.nio.charset.StandardCharsets;
 
-public class CastDateToString extends LongToStringUnaryUDF {
+/**
+ * Vectorized UDF for CAST (<DATE> TO STRING WITH FORMAT <STRING>).
+ */
+public class CastDateToStringWithFormat extends CastDateToString {
   private static final long serialVersionUID = 1L;
-  protected transient Date dt = new Date(0);
-  private transient SimpleDateFormat formatter;
+  private HiveSqlDateTimeFormatter sqlFormatter;
 
-  public CastDateToString() {
+  public CastDateToStringWithFormat() {
     super();
-    formatter = new SimpleDateFormat("yyyy-MM-dd");
-    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
   }
 
-  public CastDateToString(int inputColumn, int outputColumnNum) {
+  public CastDateToStringWithFormat(int inputColumn, byte[] patternBytes, int outputColumnNum) {
     super(inputColumn, outputColumnNum);
-    formatter = new SimpleDateFormat("yyyy-MM-dd");
-    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
-  }
 
-  // The assign method will be overridden for CHAR and VARCHAR.
-  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
-    outV.setVal(i, bytes, 0, length);
+    if (patternBytes == null) {
+      throw new IllegalStateException("Tried to cast (<date> to string with format <pattern>),"
+          + " but <pattern> not found");
+    }
+    sqlFormatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), false);
   }
 
   @Override
   protected void func(BytesColumnVector outV, long[] vector, int i) {
-    dt.setTime(DateWritableV2.daysToMillis((int) vector[i]));
-    byte[] temp = formatter.format(dt).getBytes();
-    assign(outV, i, temp, temp.length);
+    super.sqlFormat(outV, vector, i, sqlFormatter);
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return super.vectorExpressionParameters() + ", format pattern: " + sqlFormatter.getPattern();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarCharWithFormat.java
similarity index 50%
copy from ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
copy to ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarCharWithFormat.java
index dfa9f8a..6ed44a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDateToVarCharWithFormat.java
@@ -18,39 +18,40 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 
-import java.sql.Date;
-import java.text.SimpleDateFormat;
-import java.util.TimeZone;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<DATE> TO VARCHAR(<LENGTH>) WITH FORMAT <STRING>).
+ */
+public class CastDateToVarCharWithFormat extends CastDateToVarChar {
 
-public class CastDateToString extends LongToStringUnaryUDF {
   private static final long serialVersionUID = 1L;
-  protected transient Date dt = new Date(0);
-  private transient SimpleDateFormat formatter;
+  private HiveSqlDateTimeFormatter sqlFormatter;
 
-  public CastDateToString() {
+  public CastDateToVarCharWithFormat() {
     super();
-    formatter = new SimpleDateFormat("yyyy-MM-dd");
-    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
   }
 
-  public CastDateToString(int inputColumn, int outputColumnNum) {
+  public CastDateToVarCharWithFormat(int inputColumn, byte[] patternBytes, int len,
+      int outputColumnNum) {
     super(inputColumn, outputColumnNum);
-    formatter = new SimpleDateFormat("yyyy-MM-dd");
-    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+    if (patternBytes == null) {
+      throw new IllegalStateException(
+          "Tried to cast (<date> to varchar with format <pattern>)," + " but <pattern> not found");
+    }
+    sqlFormatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), false);
   }
 
-  // The assign method will be overridden for CHAR and VARCHAR.
-  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
-    outV.setVal(i, bytes, 0, length);
+  @Override protected void func(BytesColumnVector outV, long[] vector, int i) {
+    super.sqlFormat(outV, vector, i, sqlFormatter);
   }
 
-  @Override
-  protected void func(BytesColumnVector outV, long[] vector, int i) {
-    dt.setTime(DateWritableV2.daysToMillis((int) vector[i]));
-    byte[] temp = formatter.format(dt).getBytes();
-    assign(outV, i, temp, temp.length);
+  @Override public String vectorExpressionParameters() {
+    return super.vectorExpressionParameters() + ", format pattern: " + sqlFormatter.getPattern();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
index a6dff12..41ad779 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDate.java
@@ -151,14 +151,17 @@ public class CastStringToDate extends VectorExpression {
     }
   }
 
-  private void evaluate(LongColumnVector outputColVector, BytesColumnVector inV, int i) {
+  protected void evaluate(LongColumnVector outputColVector, BytesColumnVector inV, int i) {
     String dateString = new String(inV.vector[i], inV.start[i], inV.length[i], StandardCharsets.UTF_8);
     Date hDate = new Date();
     if (dateParser.parseDate(dateString, hDate)) {
       outputColVector.vector[i] = DateWritableV2.dateToDays(hDate);
       return;
     }
+    setNull(outputColVector, i);
+  }
 
+  protected void setNull(LongColumnVector outputColVector, int i) {
     outputColVector.vector[i] = 1;
     outputColVector.isNull[i] = true;
     outputColVector.noNulls = false;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDateWithFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDateWithFormat.java
new file mode 100644
index 0000000..b50ed0e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDateWithFormat.java
@@ -0,0 +1,63 @@
+/*
+ * 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 org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<STRING> TO DATE WITH FORMAT <STRING>).
+ */
+public class CastStringToDateWithFormat extends CastStringToDate {
+
+  private static final long serialVersionUID = 1L;
+  private HiveSqlDateTimeFormatter formatter;
+
+  public CastStringToDateWithFormat() {
+    super();
+  }
+
+  public CastStringToDateWithFormat(int inputColumn, byte[] patternBytes, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+
+    if (patternBytes == null) {
+      throw new IllegalStateException(
+          "Tried to cast (<string> to date with format <pattern>)," + " but <pattern> not found");
+    }
+    formatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), true);
+  }
+
+  @Override protected void evaluate(LongColumnVector outputColVector, BytesColumnVector inV,
+      int i) {
+    String dateString =
+        new String(inV.vector[i], inV.start[i], inV.length[i], StandardCharsets.UTF_8);
+    Date date = formatter.parseDate(dateString.replaceAll("\u0000", ""));
+    if (date != null) {
+      outputColVector.vector[i] = DateWritableV2.dateToDays(date);
+    } else {
+      super.setNull(outputColVector, i);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
index b48b013..68110ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
@@ -143,7 +143,7 @@ public class CastStringToTimestamp extends VectorExpression {
     }
   }
 
-  private void evaluate(TimestampColumnVector outputColVector, BytesColumnVector inputColVector, int i) {
+  protected void evaluate(TimestampColumnVector outputColVector, BytesColumnVector inputColVector, int i) {
     try {
       org.apache.hadoop.hive.common.type.Timestamp timestamp =
           PrimitiveObjectInspectorUtils.getTimestampFromString(
@@ -152,12 +152,16 @@ public class CastStringToTimestamp extends VectorExpression {
                   "UTF-8"));
       outputColVector.set(i, timestamp.toSqlTimestamp());
     } catch (Exception e) {
-      outputColVector.setNullValue(i);
-      outputColVector.isNull[i] = true;
-      outputColVector.noNulls = false;
+      setNull(outputColVector, i);
     }
   }
 
+  void setNull(TimestampColumnVector outputColVector, int i) {
+    outputColVector.setNullValue(i);
+    outputColVector.isNull[i] = true;
+    outputColVector.noNulls = false;
+  }
+
   @Override
   public String vectorExpressionParameters() {
     return getColumnParamString(0, inputColumn);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestampWithFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestampWithFormat.java
new file mode 100644
index 0000000..9361e77
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestampWithFormat.java
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<STRING> TO TIMESTAMP WITH FORMAT <STRING>).
+ */
+public class CastStringToTimestampWithFormat extends CastStringToTimestamp {
+
+  private static final long serialVersionUID = 1L;
+  private HiveSqlDateTimeFormatter formatter;
+
+  public CastStringToTimestampWithFormat() {
+    super();
+  }
+
+  public CastStringToTimestampWithFormat(int inputColumn, byte[] patternBytes,
+      int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+
+    if (patternBytes == null) {
+      throw new IllegalStateException("Tried to cast (<string> to timestamp with format"
+          + "<pattern>), but <pattern> not found");
+    }
+    formatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), true);
+  }
+
+  @Override protected void evaluate(TimestampColumnVector outputColVector,
+      BytesColumnVector inputColVector, int i) {
+    String inputString =
+        new String(inputColVector.vector[i], inputColVector.start[i], inputColVector.length[i],
+            StandardCharsets.UTF_8);
+    Timestamp timestamp = formatter.parseTimestamp(inputString.replaceAll("\u0000", ""));
+    if (timestamp != null) {
+      outputColVector.set(i, timestamp.toSqlTimestamp());
+    } else {
+      super.setNull(outputColVector, i);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToCharWithFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToCharWithFormat.java
new file mode 100644
index 0000000..5472a7e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToCharWithFormat.java
@@ -0,0 +1,58 @@
+/*
+ * 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 org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<TIMESTAMP> TO CHAR(<LENGTH>) WITH FORMAT <STRING>).
+ */
+public class CastTimestampToCharWithFormat extends CastTimestampToChar {
+
+  private static final long serialVersionUID = 1L;
+  private HiveSqlDateTimeFormatter sqlFormatter;
+
+  public CastTimestampToCharWithFormat() {
+    super();
+  }
+
+  public CastTimestampToCharWithFormat(int inputColumn, byte[] patternBytes, int len,
+      int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+
+    if (patternBytes == null) {
+      throw new IllegalStateException("Tried to cast (<timestamp> to char with format <pattern>),"
+          + " but <pattern> not found");
+    }
+    sqlFormatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), false);
+  }
+
+  @Override protected void func(BytesColumnVector outV, TimestampColumnVector inV, int i) {
+    super.sqlFormat(outV, inV, i, sqlFormatter);
+  }
+
+  @Override public String vectorExpressionParameters() {
+    return super.vectorExpressionParameters() + ", format pattern: " + sqlFormatter.getPattern();
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToString.java
index adc3a9d..58e5d45 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToString.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 
@@ -63,6 +64,22 @@ public class CastTimestampToString extends TimestampToStringUnaryUDF {
     assign(outV, i, temp, temp.length);
   }
 
+  /**
+   * CastTimestampToString, CastTimestampToChar, CastTimestampToVarchar use this.
+   */
+  void sqlFormat(BytesColumnVector outV, TimestampColumnVector inV, int i,
+      HiveSqlDateTimeFormatter sqlFormatter) {
+    String formattedString = sqlFormatter.format(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(inV.time[i], inV.nanos[i]));
+    if (formattedString == null) {
+      outV.isNull[i] = true;
+      outV.noNulls = false;
+      return;
+    }
+    byte[] temp = formattedString.getBytes();
+    assign(outV, i, temp, temp.length);
+  }
+
   public static String getTimestampString(Timestamp ts) {
     return
         LocalDateTime.ofInstant(Instant.ofEpochMilli(ts.getTime()), ZoneOffset.UTC)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToStringWithFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToStringWithFormat.java
new file mode 100644
index 0000000..148995d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToStringWithFormat.java
@@ -0,0 +1,57 @@
+/*
+ * 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 org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<TIMESTAMP> TO STRING WITH FORMAT <STRING>).
+ */
+public class CastTimestampToStringWithFormat extends CastTimestampToString {
+  private static final long serialVersionUID = 1L;
+  private HiveSqlDateTimeFormatter sqlFormatter;
+
+  public CastTimestampToStringWithFormat() {
+    super();
+  }
+
+  public CastTimestampToStringWithFormat(int inputColumn, byte[] patternBytes,
+      int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+
+    if (patternBytes == null) {
+      throw new IllegalStateException("Tried to cast (<timestamp> to string with format"
+          + " <pattern>), but <pattern> not found");
+    }
+    sqlFormatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), false);
+  }
+
+  @Override protected void func(BytesColumnVector outV, TimestampColumnVector inV, int i) {
+    super.sqlFormat(outV, inV, i, sqlFormatter);
+  }
+
+  @Override public String vectorExpressionParameters() {
+    return super.vectorExpressionParameters() + ", format pattern: " + sqlFormatter.getPattern();
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToVarCharWithFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToVarCharWithFormat.java
new file mode 100644
index 0000000..a0569e1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToVarCharWithFormat.java
@@ -0,0 +1,58 @@
+/*
+ * 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 org.apache.hadoop.hive.common.format.datetime.HiveSqlDateTimeFormatter;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Vectorized UDF for CAST (<TIMESTAMP> TO VARCHAR(<LENGTH>) WITH FORMAT <STRING>).
+ */
+public class CastTimestampToVarCharWithFormat extends CastTimestampToVarChar {
+
+  private static final long serialVersionUID = 1L;
+  private HiveSqlDateTimeFormatter sqlFormatter;
+
+  public CastTimestampToVarCharWithFormat() {
+    super();
+  }
+
+  public CastTimestampToVarCharWithFormat(int inputColumn, byte[] patternBytes, int len,
+      int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+
+    if (patternBytes == null) {
+      throw new IllegalStateException("Tried to cast (<timestamp> to varchar with format"
+          + "<pattern>), but <pattern> not found");
+    }
+    sqlFormatter =
+        new HiveSqlDateTimeFormatter(new String(patternBytes, StandardCharsets.UTF_8), false);
+  }
+
+  @Override protected void func(BytesColumnVector outV, TimestampColumnVector inV, int i) {
+    super.sqlFormat(outV, inV, i, sqlFormatter);
+  }
+
+  @Override public String vectorExpressionParameters() {
+    return super.vectorExpressionParameters() + ", format pattern: " + sqlFormatter.getPattern();
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java
index 16742ee..81540ba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCastFormat.java
@@ -190,13 +190,21 @@ public class GenericUDFCastFormat extends GenericUDF implements Serializable {
 
     // format here
     Object formattedOutput = null;
-    if (inputOI.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.DATE) {
-      formattedOutput = formatter.format((Date) input);
-      if (formattedOutput == null) {
-        return null;
+    if (inputOI.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.DATE
+        || inputOI.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP) {
+      if (inputOI.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.DATE) {
+        try {
+          formattedOutput = formatter.format((Date) input);
+        } catch (IllegalArgumentException e) {
+          return null;
+        }
+      } else {
+        try {
+          formattedOutput = formatter.format((Timestamp) input);
+        } catch (IllegalArgumentException e) {
+          return null;
+        }
       }
-    } else if (inputOI.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP) {
-      formattedOutput = formatter.format((Timestamp) input);
       if (formattedOutput == null) {
         return null;
       }
@@ -213,17 +221,26 @@ public class GenericUDFCastFormat extends GenericUDF implements Serializable {
       return ((SettableHiveVarcharObjectInspector) outputOI)
           .create(new HiveVarchar((String) formattedOutput, -1));
     case TIMESTAMP:
-      Timestamp t = formatter.parseTimestamp((String) input);
-      if (t == null) {
+      try {
+        Timestamp t = formatter.parseTimestamp((String) input);
+        if (t == null) {
+          return null;
+        }
+        return ((SettableTimestampObjectInspector) outputOI).create(t);
+      } catch (IllegalArgumentException e) {
         return null;
       }
-      return ((SettableTimestampObjectInspector) outputOI).create(t);
+
     case DATE:
-      Date d = formatter.parseDate((String) input);
-      if (d == null) {
+      try {
+        Date d = formatter.parseDate((String) input);
+        if (d == null) {
+          return null;
+        }
+        return ((SettableDateObjectInspector) outputOI).create(d);
+      } catch (IllegalArgumentException e) {
         return null;
       }
-      return ((SettableDateObjectInspector) outputOI).create(d);
     default:
       throw new HiveException("Output type " + outputOI.getPrimitiveCategory() + " not valid");
     }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
index 6632377..092d320 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
@@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Random;
 
+import org.apache.hadoop.hive.common.type.Date;
 import org.junit.Assert;
 
 import org.apache.hadoop.hive.serde2.RandomTypeUtil;
@@ -115,18 +116,20 @@ public class TestVectorMathFunctions {
     Assert.assertEquals(1.2346d, resultV.vector[7], Double.MIN_VALUE);
   }
 
-  static int DAYS_LIMIT = 365 * 9999;
+  private static final int DAYS_LIMIT = 365 * 9999;
+  //approximate, so we get some negative values:
+  private static final int SMALLEST_EPOCH_DAY = -365 * 1969;
 
   public static VectorizedRowBatch getVectorizedRowBatchDateInTimestampOut(int[] intValues) {
     Random r = new Random(12099);
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     LongColumnVector inV;
     TimestampColumnVector outV;
-    inV = new LongColumnVector();
-    outV = new TimestampColumnVector();
+    inV = new LongColumnVector(intValues.length);
+    outV = new TimestampColumnVector(intValues.length);
 
     for (int i = 0; i < intValues.length; i++) {
-      intValues[i] = r.nextInt() % DAYS_LIMIT;
+      intValues[i] = SMALLEST_EPOCH_DAY + r.nextInt() % DAYS_LIMIT;
       inV.vector[i] = intValues[i];
     }
 
@@ -137,6 +140,36 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
+  public static VectorizedRowBatch getVectorizedRowBatchDateInStringOut(int[] intValues) {
+    // get date in timestamp out, and change timestamp out to string out
+    VectorizedRowBatch batch =  getVectorizedRowBatchDateInTimestampOut(intValues);
+    BytesColumnVector outV = new BytesColumnVector(intValues.length);
+    batch.cols[1] = outV;
+    return batch;
+  }
+
+  // For testing CastDateToStringWithFormat with
+  // TestVectorTypeCastsWithFormat#testCastDateToStringWithFormat
+  public static VectorizedRowBatch getVectorizedRowBatchDateInStringOutFormatted() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    LongColumnVector dateColumnV;
+    BytesColumnVector stringColumnV;
+    dateColumnV = new LongColumnVector();
+    stringColumnV = new BytesColumnVector();
+
+    dateColumnV.vector[0] = Date.valueOf("2019-12-31").toEpochDay();
+    dateColumnV.vector[1] = Date.valueOf("1776-07-04").toEpochDay();
+    dateColumnV.vector[2] = Date.valueOf("2012-02-29").toEpochDay();
+    dateColumnV.vector[3] = Date.valueOf("1580-08-08").toEpochDay();
+    dateColumnV.vector[4] = Date.valueOf("0005-01-01").toEpochDay();
+    dateColumnV.vector[5] = Date.valueOf("9999-12-31").toEpochDay();
+
+    batch.cols[0] = dateColumnV;
+    batch.cols[1] = stringColumnV;
+    batch.size = 6;
+    return batch;
+  }
+
   public static VectorizedRowBatch getVectorizedRowBatchDoubleInLongOut() {
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     LongColumnVector lcv;
@@ -277,6 +310,42 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
+  public static VectorizedRowBatch getVectorizedRowBatchStringInTimestampOutFormatted() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    BytesColumnVector inV;
+    inV = new BytesColumnVector();
+    inV.initBuffer();
+    inV.setVal(0, StandardCharsets.UTF_8.encode("2019-12-31 00:00:00.999999999").array());
+    inV.setVal(1, StandardCharsets.UTF_8.encode("1776-07-04 17:07:06.177617761").array());
+    inV.setVal(2, StandardCharsets.UTF_8.encode("2012-02-29 23:59:59.999999999").array());
+    inV.setVal(3, StandardCharsets.UTF_8.encode("1580-08-08 00:00:00.0").array());
+    inV.setVal(4, StandardCharsets.UTF_8.encode("0005-01-01 00:00:00.0").array());
+    inV.setVal(5, StandardCharsets.UTF_8.encode("9999-12-31 23:59:59.999999999").array());
+
+    batch.cols[0] = inV;
+
+    batch.size = 6;
+    return batch;
+  }
+
+  public static VectorizedRowBatch getVectorizedRowBatchStringInDateOutFormatted() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    BytesColumnVector inV;
+    inV = new BytesColumnVector();
+    inV.initBuffer();
+    inV.setVal(0, StandardCharsets.UTF_8.encode("19/12/31").array());
+    inV.setVal(1, StandardCharsets.UTF_8.encode("1776--07--04").array());
+    inV.setVal(2, StandardCharsets.UTF_8.encode("2012/02/29").array());
+    inV.setVal(3, StandardCharsets.UTF_8.encode("1580/08/08").array());
+    inV.setVal(4, StandardCharsets.UTF_8.encode("0005/01/01").array());
+    inV.setVal(5, StandardCharsets.UTF_8.encode("9999/12/31").array());
+
+    batch.cols[0] = inV;
+
+    batch.size = 6;
+    return batch;
+  }
+
   public static VectorizedRowBatch getVectorizedRowBatchTimestampInLongOut(long[] longValues) {
     Random r = new Random(345);
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
@@ -297,6 +366,55 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
+
+  public static VectorizedRowBatch getVectorizedRowBatchTimestampInStringOut(
+      long[] epochSecondValues, int[] nanoValues) {
+    Random r = new Random(345);
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    batch.size = epochSecondValues.length;
+
+    TimestampColumnVector inV;
+    BytesColumnVector outV;
+    inV = new TimestampColumnVector(batch.size);
+    outV = new BytesColumnVector(batch.size);
+
+    for (int i = 0; i < batch.size; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      epochSecondValues[i] = randTimestamp.toEpochSecond();
+      nanoValues[i] = randTimestamp.getNanos();
+      inV.set(i, randTimestamp.toSqlTimestamp());
+    }
+
+    batch.cols[0] = inV;
+    batch.cols[1] = outV;
+
+    return batch;
+  }
+
+  public static VectorizedRowBatch getVectorizedRowBatchTimestampInStringOutFormatted() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    TimestampColumnVector timestampColumnV;
+    BytesColumnVector stringColumnV;
+    timestampColumnV = new TimestampColumnVector();
+    stringColumnV = new BytesColumnVector();
+
+    timestampColumnV.set(0, getSqlTimestamp("2019-12-31 19:20:21.999999999"));
+    timestampColumnV.set(1, getSqlTimestamp("1776-07-04 17:07:06.177617761"));
+    timestampColumnV.set(2, getSqlTimestamp("2012-02-29 23:59:59.999999999"));
+    timestampColumnV.set(3, getSqlTimestamp("1580-08-08 00:00:00"));
+    timestampColumnV.set(4, getSqlTimestamp("0005-01-01 00:00:00"));
+    timestampColumnV.set(5, getSqlTimestamp("9999-12-31 23:59:59.999999999"));
+
+    batch.cols[0] = timestampColumnV;
+    batch.cols[1] = stringColumnV;
+    batch.size = 6;
+    return batch;
+  }
+
+  private static java.sql.Timestamp getSqlTimestamp(String s) {
+    return Timestamp.valueOf(s).toSqlTimestamp();
+  }
+
   static long SECONDS_LIMIT = 60L * 24L * 365L * 9999L;
 
   public static VectorizedRowBatch getVectorizedRowBatchLongInTimestampOut(long[] longValues) {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 58fd7b0..a449ea1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -23,7 +23,9 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
 import java.sql.Timestamp;
+import java.util.Arrays;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
@@ -72,6 +74,30 @@ public class TestVectorTypeCasts {
     Assert.assertEquals(1, resultV.vector[6]);
   }
 
+  // +8 hours from PST to GMT, needed because java.sql.Date will subtract 8 hours from final
+  // value because VM in test time zone is PST.
+  private static final long TIME_DIFFERENCE = 28800000L;
+  @Test
+  public void testCastDateToString() throws HiveException {
+    int[] intValues = new int[100];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDateInStringOut(intValues);
+    BytesColumnVector resultV = (BytesColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new CastDateToString(0, 1);
+    expr.evaluate(b);
+
+    String expected, result;
+    for (int i = 0; i < intValues.length; i++) {
+      expected =
+          new java.sql.Date(DateWritableV2.daysToMillis(intValues[i]) + TIME_DIFFERENCE).toString();
+      byte[] subbyte = Arrays.copyOfRange(resultV.vector[i], resultV.start[i],
+          resultV.start[i] + resultV.length[i]);
+      result = new String(subbyte, StandardCharsets.UTF_8);
+
+      Assert.assertEquals("Index: " + i + " Epoch day value: " + intValues[i], expected, result);
+    }
+  }
+
   @Test
   public void testCastDateToTimestamp() throws HiveException {
     int[] intValues = new int[500];
@@ -192,6 +218,31 @@ public class TestVectorTypeCasts {
     }
   }
 
+  @Test
+  public void testCastTimestampToString() throws HiveException {
+    int numberToTest = 100;
+    long[] epochSecondValues = new long[numberToTest];
+    int[] nanoValues = new int[numberToTest];
+    VectorizedRowBatch b =
+        TestVectorMathFunctions.getVectorizedRowBatchTimestampInStringOut(epochSecondValues, nanoValues);
+    BytesColumnVector resultV = (BytesColumnVector) b.cols[1];
+    b.cols[0].noNulls = true;
+    VectorExpression expr = new CastTimestampToString(0, 1);
+    expr.evaluate(b);
+
+    String expected, result;
+    for (int i = 0; i < numberToTest; i++) {
+      expected = org.apache.hadoop.hive.common.type.Timestamp
+          .ofEpochSecond(epochSecondValues[i], nanoValues[i]).toString();
+      byte[] subbyte = Arrays.copyOfRange(resultV.vector[i], resultV.start[i],
+          resultV.start[i] + resultV.length[i]);
+      result = new String(subbyte, StandardCharsets.UTF_8);
+      Assert.assertEquals("Index: " +  i + " Seconds since epoch: " + epochSecondValues[i] +
+              " nanoseconds: " + nanoValues[i],
+          expected, result);
+    }
+  }
+
   public byte[] toBytes(String s) {
     byte[] b = null;
     try {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCastsWithFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCastsWithFormat.java
new file mode 100644
index 0000000..533657d
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCastsWithFormat.java
@@ -0,0 +1,147 @@
+/*
+ * 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 org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Tests vectorized type cast udfs CastDateToStringWithFormat, CastTimestampToStringWithFormat,
+ * CastStringToDateWithFormat, CastStringToTimestampWithFormat.
+ */
+public class TestVectorTypeCastsWithFormat {
+
+  @Test
+  public void testCastDateToStringWithFormat() throws HiveException {
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDateInStringOutFormatted();
+    BytesColumnVector resultV = (BytesColumnVector) b.cols[1];
+    VectorExpression expr = new CastDateToStringWithFormat(0, "yyyy".getBytes(), 1);
+    expr.evaluate(b);
+    verifyString(0, "2019", resultV);
+    verifyString(1, "1776", resultV);
+    verifyString(2, "2012", resultV);
+    verifyString(3, "1580", resultV);
+    verifyString(4, "0005", resultV);
+    verifyString(5, "9999", resultV);
+
+    expr = new CastDateToStringWithFormat(0, "MM".getBytes(), 1);
+    resultV = new BytesColumnVector();
+    b.cols[1] = resultV;
+    expr.evaluate(b);
+    verifyString(0, "12", resultV);
+    verifyString(1, "07", resultV);
+    verifyString(2, "02", resultV);
+    verifyString(3, "08", resultV);
+    verifyString(4, "01", resultV);
+    verifyString(5, "12", resultV);
+  }
+
+  @Test
+  public void testCastTimestampToStringWithFormat() throws HiveException {
+    VectorizedRowBatch b =
+        TestVectorMathFunctions.getVectorizedRowBatchTimestampInStringOutFormatted();
+    BytesColumnVector resultV = (BytesColumnVector) b.cols[1];
+    VectorExpression expr = new CastTimestampToStringWithFormat(0, "yyyy".getBytes(), 1);
+    expr.evaluate(b);
+
+    Assert.assertEquals("2019", getStringFromBytesColumnVector(resultV, 0));
+    Assert.assertEquals("1776", getStringFromBytesColumnVector(resultV, 1));
+    Assert.assertEquals("2012", getStringFromBytesColumnVector(resultV, 2));
+    Assert.assertEquals("1580", getStringFromBytesColumnVector(resultV, 3));
+    Assert.assertEquals("0005", getStringFromBytesColumnVector(resultV, 4));
+    Assert.assertEquals("9999", getStringFromBytesColumnVector(resultV, 5));
+
+    resultV = new BytesColumnVector();
+    b.cols[1] = resultV;
+    expr = new CastTimestampToStringWithFormat(0, "HH24".getBytes(), 1);
+    expr.evaluate(b);
+
+    Assert.assertEquals("19", getStringFromBytesColumnVector(resultV, 0));
+    Assert.assertEquals("17", getStringFromBytesColumnVector(resultV, 1));
+    Assert.assertEquals("23", getStringFromBytesColumnVector(resultV, 2));
+    Assert.assertEquals("00", getStringFromBytesColumnVector(resultV, 3));
+    Assert.assertEquals("00", getStringFromBytesColumnVector(resultV, 4));
+    Assert.assertEquals("23", getStringFromBytesColumnVector(resultV, 5));
+  }
+
+  @Test
+  public void testCastStringToTimestampWithFormat() throws HiveException {
+    VectorizedRowBatch b =
+        TestVectorMathFunctions.getVectorizedRowBatchStringInTimestampOutFormatted();
+    TimestampColumnVector resultV;
+    resultV = new TimestampColumnVector();
+    b.cols[1] = resultV;
+    VectorExpression expr =
+        new CastStringToTimestampWithFormat(0, "yyyy.mm.dd HH24.mi.ss.ff".getBytes(), 1);
+    expr.evaluate(b);
+
+    verifyTimestamp("2019-12-31 00:00:00.999999999", resultV, 0);
+    verifyTimestamp("1776-07-04 17:07:06.177617761", resultV, 1);
+    verifyTimestamp("2012-02-29 23:59:59.999999999", resultV, 2);
+    verifyTimestamp("1580-08-08 00:00:00", resultV, 3);
+    verifyTimestamp("0005-01-01 00:00:00", resultV, 4);
+    verifyTimestamp("9999-12-31 23:59:59.999999999", resultV, 5);
+  }
+
+  private void verifyTimestamp(String tsString, TimestampColumnVector resultV, int index) {
+    Assert.assertEquals(Timestamp.valueOf(tsString).toEpochMilli(), resultV.time[index]);
+    Assert.assertEquals(Timestamp.valueOf(tsString).getNanos(), resultV.nanos[index]);
+  }
+
+  @Test
+  public void testCastStringToDateWithFormat() throws HiveException {
+    VectorizedRowBatch b =
+        TestVectorMathFunctions.getVectorizedRowBatchStringInDateOutFormatted();
+    LongColumnVector resultV;
+    resultV = new LongColumnVector();
+    b.cols[1] = resultV;
+    VectorExpression expr = new CastStringToDateWithFormat(0, "yyyy.mm.dd".getBytes(), 1);
+    expr.evaluate(b);
+
+    Assert.assertEquals(Date.valueOf("2019-12-31").toEpochDay(), resultV.vector[0]);
+    Assert.assertEquals(Date.valueOf("1776-07-04").toEpochDay(), resultV.vector[1]);
+    Assert.assertEquals(Date.valueOf("2012-02-29").toEpochDay(), resultV.vector[2]);
+    Assert.assertEquals(Date.valueOf("1580-08-08").toEpochDay(), resultV.vector[3]);
+    Assert.assertEquals(Date.valueOf("0005-01-01").toEpochDay(), resultV.vector[4]);
+    Assert.assertEquals(Date.valueOf("9999-12-31").toEpochDay(), resultV.vector[5]);
+  }
+
+  private void verifyString(int resultIndex, String expected, BytesColumnVector resultV) {
+    String result = getStringFromBytesColumnVector(resultV, resultIndex);
+    Assert.assertEquals(expected, result);
+  }
+
+  private String getStringFromBytesColumnVector(BytesColumnVector resultV, int i) {
+    String result;
+    byte[] resultBytes = Arrays.copyOfRange(resultV.vector[i], resultV.start[i],
+        resultV.start[i] + resultV.length[i]);
+    result = new String(resultBytes, StandardCharsets.UTF_8);
+    return result;
+  }
+}
diff --git a/ql/src/test/queries/clientnegative/udf_cast_format_bad_pattern.q b/ql/src/test/queries/clientnegative/udf_cast_format_bad_pattern.q
new file mode 100644
index 0000000..649d937
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/udf_cast_format_bad_pattern.q
@@ -0,0 +1 @@
+select cast ("2015-05-15 12:01:00" as date format "x");
diff --git a/ql/src/test/queries/clientpositive/cast_datetime_with_sql_2016_format.q b/ql/src/test/queries/clientpositive/cast_datetime_with_sql_2016_format.q
index 269edf6..5a2a6d7 100644
--- a/ql/src/test/queries/clientpositive/cast_datetime_with_sql_2016_format.q
+++ b/ql/src/test/queries/clientpositive/cast_datetime_with_sql_2016_format.q
@@ -37,9 +37,30 @@ from varchars   select cast (s as date      format "yyyy.mm.dd");
 from chars      select cast (s as timestamp format "yyyy.mm.dd");
 from chars      select cast (s as date      format "yyyy.mm.dd");
 
+--output null when input can't be parsed
+select cast ("2015-05-15 12::00" as timestamp format "yyyy-mm-dd hh:mi:ss");
+select cast ("2015-05-15 12::00" as date format "yyyy-mm-dd hh:mi:ss");
 
 --correct descriptions
 explain from strings    select cast (s as timestamp   format "yyy.mm.dd");
 explain from strings    select cast (s as date        format "yyy.mm.dd");
 explain from timestamp1 select cast (t as string      format "yyyy");
 explain from timestamp1 select cast (t as varchar(12) format "yyyy");
+
+
+--vectorized
+set hive.vectorized.execution.enabled=true;
+set hive.fetch.task.conversion=none;
+
+from timestamp1 select cast (t as string      format "yyyy");
+from dates      select cast (d as string      format "yyyy");
+from timestamp1 select cast (t as varchar(11) format "yyyy");
+from dates      select cast (d as varchar(11) format "yyyy");
+from timestamp1 select cast (t as char(11)    format "yyyy");
+from dates      select cast (d as char(11)    format "yyyy");
+from strings    select cast (s as timestamp   format "yyyy.mm.dd");
+from varchars   select cast (s as timestamp   format "yyyy.mm.dd");
+from chars      select cast (s as timestamp   format "yyyy.mm.dd");
+from strings    select cast (s as date        format "yyyy.mm.dd");
+from varchars   select cast (s as date        format "yyyy.mm.dd");
+from chars      select cast (s as date        format "yyyy.mm.dd");
diff --git a/ql/src/test/results/clientnegative/udf_cast_format_bad_pattern.q.out b/ql/src/test/results/clientnegative/udf_cast_format_bad_pattern.q.out
new file mode 100644
index 0000000..7580e7b
--- /dev/null
+++ b/ql/src/test/results/clientnegative/udf_cast_format_bad_pattern.q.out
@@ -0,0 +1 @@
+FAILED: IllegalArgumentException Bad date/time conversion pattern: x
diff --git a/ql/src/test/results/clientpositive/cast_datetime_with_sql_2016_format.q.out b/ql/src/test/results/clientpositive/cast_datetime_with_sql_2016_format.q.out
index 4a502b9..e1fd341 100644
--- a/ql/src/test/results/clientpositive/cast_datetime_with_sql_2016_format.q.out
+++ b/ql/src/test/results/clientpositive/cast_datetime_with_sql_2016_format.q.out
@@ -227,6 +227,24 @@ POSTHOOK: Input: default@chars
 #### A masked pattern was here ####
 2020-02-03
 1969-12-31
+PREHOOK: query: select cast ("2015-05-15 12::00" as timestamp format "yyyy-mm-dd hh:mi:ss")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select cast ("2015-05-15 12::00" as timestamp format "yyyy-mm-dd hh:mi:ss")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+NULL
+PREHOOK: query: select cast ("2015-05-15 12::00" as date format "yyyy-mm-dd hh:mi:ss")
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select cast ("2015-05-15 12::00" as date format "yyyy-mm-dd hh:mi:ss")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+NULL
 PREHOOK: query: explain from strings    select cast (s as timestamp   format "yyy.mm.dd")
 PREHOOK: type: QUERY
 PREHOOK: Input: default@strings
@@ -327,3 +345,123 @@ STAGE PLANS:
             Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
+PREHOOK: query: from timestamp1 select cast (t as string      format "yyyy")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp1
+#### A masked pattern was here ####
+POSTHOOK: query: from timestamp1 select cast (t as string      format "yyyy")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp1
+#### A masked pattern was here ####
+2020
+1969
+PREHOOK: query: from dates      select cast (d as string      format "yyyy")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dates
+#### A masked pattern was here ####
+POSTHOOK: query: from dates      select cast (d as string      format "yyyy")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dates
+#### A masked pattern was here ####
+2020
+1969
+PREHOOK: query: from timestamp1 select cast (t as varchar(11) format "yyyy")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp1
+#### A masked pattern was here ####
+POSTHOOK: query: from timestamp1 select cast (t as varchar(11) format "yyyy")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp1
+#### A masked pattern was here ####
+2020
+1969
+PREHOOK: query: from dates      select cast (d as varchar(11) format "yyyy")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dates
+#### A masked pattern was here ####
+POSTHOOK: query: from dates      select cast (d as varchar(11) format "yyyy")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dates
+#### A masked pattern was here ####
+2020
+1969
+PREHOOK: query: from timestamp1 select cast (t as char(11)    format "yyyy")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp1
+#### A masked pattern was here ####
+POSTHOOK: query: from timestamp1 select cast (t as char(11)    format "yyyy")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp1
+#### A masked pattern was here ####
+2020       
+1969       
+PREHOOK: query: from dates      select cast (d as char(11)    format "yyyy")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dates
+#### A masked pattern was here ####
+POSTHOOK: query: from dates      select cast (d as char(11)    format "yyyy")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dates
+#### A masked pattern was here ####
+2020       
+1969       
+PREHOOK: query: from strings    select cast (s as timestamp   format "yyyy.mm.dd")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@strings
+#### A masked pattern was here ####
+POSTHOOK: query: from strings    select cast (s as timestamp   format "yyyy.mm.dd")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@strings
+#### A masked pattern was here ####
+2020-02-03 00:00:00
+1969-12-31 00:00:00
+PREHOOK: query: from varchars   select cast (s as timestamp   format "yyyy.mm.dd")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@varchars
+#### A masked pattern was here ####
+POSTHOOK: query: from varchars   select cast (s as timestamp   format "yyyy.mm.dd")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@varchars
+#### A masked pattern was here ####
+2020-02-03 00:00:00
+1969-12-31 00:00:00
+PREHOOK: query: from chars      select cast (s as timestamp   format "yyyy.mm.dd")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@chars
+#### A masked pattern was here ####
+POSTHOOK: query: from chars      select cast (s as timestamp   format "yyyy.mm.dd")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@chars
+#### A masked pattern was here ####
+2020-02-03 00:00:00
+1969-12-31 00:00:00
+PREHOOK: query: from strings    select cast (s as date        format "yyyy.mm.dd")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@strings
+#### A masked pattern was here ####
+POSTHOOK: query: from strings    select cast (s as date        format "yyyy.mm.dd")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@strings
+#### A masked pattern was here ####
+2020-02-03
+1969-12-31
+PREHOOK: query: from varchars   select cast (s as date        format "yyyy.mm.dd")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@varchars
+#### A masked pattern was here ####
+POSTHOOK: query: from varchars   select cast (s as date        format "yyyy.mm.dd")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@varchars
+#### A masked pattern was here ####
+2020-02-03
+1969-12-31
+PREHOOK: query: from chars      select cast (s as date        format "yyyy.mm.dd")
+PREHOOK: type: QUERY
+PREHOOK: Input: default@chars
+#### A masked pattern was here ####
+POSTHOOK: query: from chars      select cast (s as date        format "yyyy.mm.dd")
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@chars
+#### A masked pattern was here ####
+2020-02-03
+1969-12-31