You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/11/02 21:21:02 UTC

[jira] [Commented] (DRILL-6768) Improve to_date, to_time and to_timestamp and corresponding cast functions to handle empty string when `drill.exec.functions.cast_empty_string_to_null` option is enabled

    [ https://issues.apache.org/jira/browse/DRILL-6768?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16673694#comment-16673694 ] 

ASF GitHub Bot commented on DRILL-6768:
---------------------------------------

sohami closed pull request #1494: DRILL-6768: Improve to_date, to_time and to_timestamp and correspondi…
URL: https://github.com/apache/drill/pull/1494
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/exec/java-exec/src/main/codegen/data/Casts.tdd b/exec/java-exec/src/main/codegen/data/Casts.tdd
index e43572a8781..31eef193ec5 100644
--- a/exec/java-exec/src/main/codegen/data/Casts.tdd
+++ b/exec/java-exec/src/main/codegen/data/Casts.tdd
@@ -61,10 +61,30 @@
     {from: "VarChar", to: "TimeStamp", major: "VarCharDate", alias: "timestamptype"},
     {from: "VarChar", to: "Time", major: "VarCharDate", alias: "timetype"},
 
+    {from: "VarChar", to: "NullableDate", major: "NullableVarCharDate"},
+    {from: "VarChar", to: "NullableTimeStamp", major: "NullableVarCharDate"},
+    {from: "VarChar", to: "NullableTime", major: "NullableVarCharDate"},
+
+    {from: "NullableVarChar", to: "NullableDate", major: "NullableVarCharDate"},
+    {from: "NullableVarChar", to: "NullableTimeStamp", major: "NullableVarCharDate"},
+    {from: "NullableVarChar", to: "NullableTime", major: "NullableVarCharDate"},
+
     {from: "VarBinary", to: "Date", major: "VarBinaryDate", alias: "datetype"},
     {from: "VarBinary", to: "TimeStamp", major: "VarBinaryDate", alias: "timestamptype"},
     {from: "VarBinary", to: "Time", major: "VarBinaryDate", alias: "timetype"},
 
+    {from: "VarBinary", to: "NullableDate", major: "NullableVarCharDate"},
+    {from: "VarBinary", to: "NullableTimeStamp", major: "NullableVarCharDate"},
+    {from: "VarBinary", to: "NullableTime", major: "NullableVarCharDate"},
+
+    {from: "NullableVarBinary", to: "NullableDate", major: "NullableVarCharDate"},
+    {from: "NullableVarBinary", to: "NullableTimeStamp", major: "NullableVarCharDate"},
+    {from: "NullableVarBinary", to: "NullableTime", major: "NullableVarCharDate"},
+
+    {from: "NullableVar16Char", to: "NullableDate", major: "NullableVarCharDate"},
+    {from: "NullableVar16Char", to: "NullableTimeStamp", major: "NullableVarCharDate"},
+    {from: "NullableVar16Char", to: "NullableTime", major: "NullableVarCharDate"},
+
     {from: "Date", to: "VarChar", major: "DateVarChar", bufferLength: "10"}
     {from: "TimeStamp", to: "VarChar", major: "DateVarChar", bufferLength: "23"},
     {from: "Time", to: "VarChar", major: "DateVarChar", bufferLength: "12"},
@@ -73,6 +93,14 @@
     {from: "VarChar", to: "IntervalDay", major: "VarCharInterval"},
     {from: "VarChar", to: "IntervalYear", major: "VarCharInterval"},
 
+    {from: "VarChar", to: "NullableInterval", major: "NullableVarCharInterval"},
+    {from: "VarChar", to: "NullableIntervalDay", major: "NullableVarCharInterval"},
+    {from: "VarChar", to: "NullableIntervalYear", major: "NullableVarCharInterval"},
+
+    {from: "NullableVarChar", to: "NullableInterval", major: "NullableVarCharInterval"},
+    {from: "NullableVarChar", to: "NullableIntervalDay", major: "NullableVarCharInterval"},
+    {from: "NullableVarChar", to: "NullableIntervalYear", major: "NullableVarCharInterval"},
+
     {from: "Interval", to: "VarChar", major: "IntervalVarChar", bufferLength: "65"},
     {from: "IntervalYear", to: "VarChar", major: "IntervalYearVarChar", bufferLength: "35"},
     {from: "IntervalDay", to: "VarChar", major: "IntervalDayVarChar", bufferLength: "43"},
@@ -118,27 +146,27 @@
     {from: "VarChar", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "VarChar", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "VarChar", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
+    {from: "VarChar", to: "NullableVarDecimal", major: "NullableVarCharDecimalComplex"},
 
     {from: "NullableVarChar", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "NullableVarChar", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "NullableVarChar", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "NullableVarChar", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "NullableVarChar", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
+    {from: "NullableVarChar", to: "NullableVarDecimal", major: "NullableVarCharDecimalComplex"},
 
     {from: "NullableVar16Char", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "NullableVar16Char", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "NullableVar16Char", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "NullableVar16Char", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "NullableVar16Char", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
+    {from: "NullableVar16Char", to: "NullableVarDecimal", major: "NullableVarCharDecimalComplex"},
 
     {from: "NullableVarBinary", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "NullableVarBinary", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "NullableVarBinary", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "NullableVarBinary", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "NullableVarBinary", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
+    {from: "NullableVarBinary", to: "NullableVarDecimal", major: "NullableVarCharDecimalComplex"},
   ]
 }
diff --git a/exec/java-exec/src/main/codegen/data/DateIntervalFunc.tdd b/exec/java-exec/src/main/codegen/data/DateIntervalFunc.tdd
index 9f1ae37b21a..96e16070aa5 100644
--- a/exec/java-exec/src/main/codegen/data/DateIntervalFunc.tdd
+++ b/exec/java-exec/src/main/codegen/data/DateIntervalFunc.tdd
@@ -18,4 +18,18 @@
     {intervals: ["Interval", "IntervalDay", "IntervalYear", "Int", "BigInt"] },
     {truncInputTypes: ["Date", "TimeStamp", "Time", "Interval", "IntervalDay", "IntervalYear"] },
     {truncUnits : ["Second", "Minute", "Hour", "Day", "Month", "Year", "Week", "Quarter", "Decade", "Century", "Millennium" ] },
+
+    {
+        varCharToDate: [
+            {from: "VarChar", to: "Date"},
+            {from: "VarChar", to: "NullableDate"},
+            {from: "NullableVarChar", to: "NullableDate"},
+            {from: "VarChar", to: "Time"},
+            {from: "VarChar", to: "NullableTime"},
+            {from: "NullableVarChar",  to: "NullableTime"},
+            {from: "VarChar", to: "TimeStamp"},
+            {from: "VarChar", to: "NullableTimeStamp"},
+            {from: "NullableVarChar", to: "NullableTimeStamp"}
+        ]
+    }
 }
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java b/exec/java-exec/src/main/codegen/templates/CastStringTypesToDate.java
similarity index 53%
rename from exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
rename to exec/java-exec/src/main/codegen/templates/CastStringTypesToDate.java
index 8d1ab7d1b7f..c26f6e242e2 100644
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
+++ b/exec/java-exec/src/main/codegen/templates/CastStringTypesToDate.java
@@ -18,9 +18,13 @@
 <@pp.dropOutputFile />
 
 <#list cast.types as type>
-<#if type.major == "VarCharDate" || type.major == "VarBinaryDate">  <#-- Template to convert from VarChar/ VarBinary to Date, Time, TimeStamp -->
+<#if type.major == "VarCharDate" || type.major == "VarBinaryDate" || type.major == "NullableVarCharDate">  <#-- Template to convert from VarChar/ VarBinary to Date, Time, TimeStamp -->
 
+<#if type.major == "VarCharDate" || type.major == "VarBinaryDate">
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}To${type.to}.java" />
+<#elseif type.major == "NullableVarCharDate">
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/GCast${type.from}To${type.to}.java" />
+</#if>
 
 <#include "/@includes/license.ftl" />
 
@@ -46,9 +50,17 @@
  * This class is generated using freemarker and the ${.template_name} template.
  */
 @SuppressWarnings("unused")
+<#if type.major == "VarCharDate" || type.major == "VarBinaryDate">
 @FunctionTemplate(names = {"cast${type.to?upper_case}", "${type.alias}"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL, 
   costCategory = FunctionCostCategory.COMPLEX)
 public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
+<#elseif type.major == "NullableVarCharDate">
+@FunctionTemplate(name = "castEmptyString${type.from}To${type.to?upper_case}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.INTERNAL,
+    isInternal = true)
+public class GCast${type.from}To${type.to} implements DrillSimpleFunc {
+</#if>
 
   @Param ${type.from}Holder in;
   @Output ${type.to}Holder out;
@@ -56,24 +68,31 @@
   public void setup() { }
 
   public void eval() {
+    <#if type.major == "NullableVarCharDate">
+    if(<#if type.from == "NullableVarChar" || type.from == "NullableVar16Char" || type.from == "NullableVarBinary">in.isSet == 0 || </#if>in.end == in.start) {
+      out.isSet = 0;
+      return;
+    }
+    out.isSet = 1;
+    </#if>
 
-      <#if type.to != "Date">
-      byte[] buf = new byte[in.end - in.start];
-      in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-      String input = new String(buf, com.google.common.base.Charsets.UTF_8);
-      </#if>  
-      
-      <#if type.to == "Date">
-      out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.getDate(in.buffer, in.start, in.end);
+    <#if type.to != "Date" && type.to != "NullableDate">
+    byte[] buf = new byte[in.end - in.start];
+    in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
+    String input = new String(buf, com.google.common.base.Charsets.UTF_8);
+    </#if>
 
-      <#elseif type.to == "TimeStamp">
-      java.time.LocalDateTime parsedDateTime = org.apache.drill.exec.expr.fn.impl.DateUtility.parseBest(input);
-      out.value = parsedDateTime.toInstant(java.time.ZoneOffset.UTC).toEpochMilli();
+    <#if type.to == "Date" || type.to == "NullableDate">
+    out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.getDate(in.buffer, in.start, in.end);
 
-      <#elseif type.to == "Time">
-      java.time.format.DateTimeFormatter f = org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeFormatter();
-      out.value = (int) (java.time.LocalTime.parse(input, f).atDate(java.time.LocalDate.ofEpochDay(0)).toInstant(java.time.ZoneOffset.UTC).toEpochMilli());
-      </#if>
+    <#elseif type.to == "TimeStamp" || type.to == "NullableTimeStamp">
+    java.time.LocalDateTime parsedDateTime = org.apache.drill.exec.expr.fn.impl.DateUtility.parseBest(input);
+    out.value = parsedDateTime.toInstant(java.time.ZoneOffset.UTC).toEpochMilli();
+
+    <#elseif type.to == "Time" || type.to == "NullableTime">
+    java.time.format.DateTimeFormatter f = org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeFormatter();
+    out.value = (int) (java.time.LocalTime.parse(input, f).atDate(java.time.LocalDate.ofEpochDay(0)).toInstant(java.time.ZoneOffset.UTC).toEpochMilli());
+    </#if>
   }
 }
 </#if> <#-- type.major -->
diff --git a/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java b/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
new file mode 100644
index 00000000000..196e8605477
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
@@ -0,0 +1,111 @@
+/*
+ * 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.
+ */
+<@pp.dropOutputFile />
+
+<#list cast.types as type>
+<#if type.major == "VarCharInterval" || type.major == "NullableVarCharInterval">  <#-- Template to convert from VarChar to Interval, IntervalYear, IntervalDay -->
+
+<#if type.major == "VarCharInterval">
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}To${type.to}.java" />
+<#elseif type.major == "NullableVarCharInterval">
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/CastEmptyString${type.from}To${type.to}.java" />
+</#if>
+
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.expr.fn.impl.gcast;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+import org.joda.time.MutableDateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.DateMidnight;
+import javax.inject.Inject;
+import io.netty.buffer.DrillBuf;
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+@SuppressWarnings("unused")
+<#if type.major == "VarCharInterval">
+@FunctionTemplate(name = "cast${type.to?upper_case}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.NULL_IF_NULL)
+public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
+<#elseif type.major == "NullableVarCharInterval">
+@FunctionTemplate(name = "castEmptyString${type.from}To${type.to?upper_case}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.INTERNAL,
+    isInternal = true)
+public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc {
+</#if>
+
+  @Param ${type.from}Holder in;
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    <#if type.major == "NullableVarCharInterval">
+    if (<#if type.from == "NullableVarChar" || type.from == "NullableVar16Char" || type.from == "NullableVarBinary">in.isSet == 0 || </#if>in.end == in.start) {
+      out.isSet = 0;
+      return;
+    }
+    out.isSet = 1;
+    </#if>
+
+    byte[] buf = new byte[in.end - in.start];
+    in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
+    String input = new String(buf, com.google.common.base.Charsets.UTF_8);
+
+    // Parse the ISO format
+    org.joda.time.Period period = org.joda.time.Period.parse(input);
+
+    <#if type.to == "Interval" || type.to == "NullableInterval">
+    out.months       = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
+
+    out.days         = period.getDays();
+
+    out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                       (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                       (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
+                       (period.getMillis());
+
+    <#elseif type.to == "IntervalDay" || type.to == "NullableIntervalDay">
+    out.days         = period.getDays();
+
+    out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                       (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                       (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
+                       (period.getMillis());
+    <#elseif type.to == "IntervalYear" || type.to == "NullableIntervalYear">
+    out.value = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
+    </#if>
+  }
+}
+</#if> <#-- type.major -->
+</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java b/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
deleted file mode 100644
index f9396a69d5a..00000000000
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.
- */
-<@pp.dropOutputFile />
-
-<#list cast.types as type>
-<#if type.major == "VarCharInterval">  <#-- Template to convert from VarChar to Interval, IntervalYear, IntervalDay -->
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}To${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.annotations.Workspace;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.joda.time.MutableDateTime;
-import org.joda.time.DateTimeZone;
-import org.joda.time.DateMidnight;
-import javax.inject.Inject;
-import io.netty.buffer.DrillBuf;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-@SuppressWarnings("unused")
-@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
-public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
-
-  @Param ${type.from}Holder in;
-  @Output ${type.to}Holder out;
-
-  public void setup() {
-  }
-
-  public void eval() {
-
-      byte[] buf = new byte[in.end - in.start];
-      in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-      String input = new String(buf, com.google.common.base.Charsets.UTF_8);
-
-      // Parse the ISO format
-      org.joda.time.Period period = org.joda.time.Period.parse(input);
-
-      <#if type.to == "Interval">
-      out.months       = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
-
-      out.days         = period.getDays();
-
-      out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                         (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                         (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                         (period.getMillis());
-
-      <#elseif type.to == "IntervalDay">
-      out.days         = period.getDays();
-
-      out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                         (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                         (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                         (period.getMillis());
-      <#elseif type.to == "IntervalYear">
-      out.value = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
-      </#if>
-  }
-}
-</#if> <#-- type.major -->
-</#list>
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/SqlToDateTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/SqlToDateTypeFunctions.java
index 32d824a980b..19c93514964 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/SqlToDateTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/SqlToDateTypeFunctions.java
@@ -19,9 +19,9 @@
 
 <@pp.dropOutputFile/>
 
-<#list dateIntervalFunc.dates as type>
+<#list dateIntervalFunc.varCharToDate as convert>
 
-<@pp.changeOutputFile name = "/org/apache/drill/exec/expr/fn/impl/SqlTo${type}.java"/>
+<@pp.changeOutputFile name = "/org/apache/drill/exec/expr/fn/impl/G${convert.from}SqlTo${convert.to}.java"/>
 
 <#include "/@includes/license.ftl"/>
 
@@ -38,16 +38,22 @@
 /*
  * This class is generated using freemarker and the ${.template_name} template.
  */
+<#if convert.to?contains("Nullable")>
+@FunctionTemplate(name = "convert${convert.from}SqlTo${convert.to}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.INTERNAL,
+    isInternal = true)
+<#else>
+@FunctionTemplate(name = "sql_to_${convert.to?lower_case}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.NULL_IF_NULL)
+</#if>
+public class G${convert.from}SqlTo${convert.to} implements DrillSimpleFunc {
 
-@FunctionTemplate(name = "sql_to_${type?lower_case}",
-                  scope = FunctionTemplate.FunctionScope.SIMPLE,
-                  nulls = NullHandling.NULL_IF_NULL)
-public class SqlTo${type} implements DrillSimpleFunc {
-
-  @Param  VarCharHolder left;
+  @Param  ${convert.from}Holder left;
   @Param  VarCharHolder right;
   @Workspace org.joda.time.format.DateTimeFormatter format;
-  @Output ${type}Holder out;
+  @Output ${convert.to}Holder out;
 
   public void setup() {
     // Get the desired output format
@@ -57,27 +63,35 @@ public void setup() {
       format = org.joda.time.format.DateTimeFormat.forPattern(pattern);
     } catch (IllegalArgumentException e) {
       throw org.apache.drill.common.exceptions.UserException.functionError(e)
-        .message("Error parsing formatter %s in %s function", formatString, "sql_to_${type?lower_case}")
+        .message("Error parsing formatter %s in %s function", formatString, "sql_to_${convert.to?lower_case}")
         .build();
     }
   }
 
   public void eval() {
+    <#if convert.to?contains("Nullable")>
+    if (<#if convert.from == "NullableVarChar">left.isSet == 0 || </#if>left.start == left.end) {
+      out.isSet = 0;
+      return;
+    }
+    out.isSet = 1;
+
+    </#if>
     // Get the input
     String input = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.getStringFromVarCharHolder(left);
     try {
-      <#if type == "Date">
+      <#if convert.to == "Date" || convert.to == "NullableDate">
       out.value = org.joda.time.DateMidnight.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
-      <#elseif type == "TimeStamp">
+      <#elseif convert.to == "TimeStamp" || convert.to == "NullableTimeStamp">
       out.value = org.joda.time.DateTime.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
-      <#elseif type == "Time">
+      <#elseif convert.to == "Time" || convert.to == "NullableTime">
       out.value = (int) format.parseDateTime(input).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
       </#if>
     } catch (IllegalArgumentException e) {
       throw org.apache.drill.common.exceptions.UserException.functionError(e)
-        .message("Error parsing date-time %s in %s function", input, "sql_to_${type?lower_case}")
+        .message("Error parsing date-time %s in %s function", input, "sql_to_${convert.to?lower_case}")
         .build();
     }
   }
 }
-</#list>
\ No newline at end of file
+</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
index 7c410e3512c..6ef196f3da4 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
@@ -19,9 +19,9 @@
 
 <@pp.dropOutputFile />
 
-<#list dateIntervalFunc.dates as type>
+<#list dateIntervalFunc.varCharToDate as convert>
 
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/GTo${type}.java" />
+<@pp.changeOutputFile name = "/org/apache/drill/exec/expr/fn/impl/G${convert.from}To${convert.to}.java" />
 
 <#include "/@includes/license.ftl" />
 
@@ -34,43 +34,56 @@
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
  */
+<#if convert.to?contains("Nullable")>
+@FunctionTemplate(name = "convert${convert.from}To${convert.to}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.INTERNAL,
+    isInternal = true)
+<#else>
+@FunctionTemplate(name = "to_${convert.to?lower_case}",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = NullHandling.NULL_IF_NULL)
+</#if>
+public class G${convert.from}To${convert.to} implements DrillSimpleFunc {
 
-@FunctionTemplate(name = "to_${type?lower_case}" , scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-public class GTo${type} implements DrillSimpleFunc {
+  @Param  ${convert.from}Holder left;
+  @Param  VarCharHolder right;
+  @Workspace org.joda.time.format.DateTimeFormatter format;
+  @Output ${convert.to}Holder out;
 
+  public void setup() {
+    // Get the desired output format
+    byte[] buf = new byte[right.end - right.start];
+    right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
+    String formatString = new String(buf, com.google.common.base.Charsets.UTF_8);
+    format = org.joda.time.format.DateTimeFormat.forPattern(formatString);
+  }
 
-    @Param  VarCharHolder left;
-    @Param  VarCharHolder right;
-    @Workspace org.joda.time.format.DateTimeFormatter format;
-    @Output ${type}Holder out;
-
-    public void setup() {
-        // Get the desired output format
-        byte[] buf = new byte[right.end - right.start];
-        right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
-        String formatString = new String(buf, com.google.common.base.Charsets.UTF_8);
-        format = org.joda.time.format.DateTimeFormat.forPattern(formatString);
+  public void eval() {
+    <#if convert.to?contains("Nullable")>
+    if (<#if convert.from == "NullableVarChar">left.isSet == 0 || </#if>left.start == left.end) {
+      out.isSet = 0;
+      return;
     }
+    out.isSet = 1;
 
-    public void eval() {
-
-        // Get the input
-        byte[] buf1 = new byte[left.end - left.start];
-        left.buffer.getBytes(left.start, buf1, 0, left.end - left.start);
-        String input = new String(buf1, com.google.common.base.Charsets.UTF_8);
+    </#if>
+    // Get the input
+    byte[] buf1 = new byte[left.end - left.start];
+    left.buffer.getBytes(left.start, buf1, 0, left.end - left.start);
+    String input = new String(buf1, com.google.common.base.Charsets.UTF_8);
 
-        <#if type == "Date">
-        out.value = (org.joda.time.DateMidnight.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC)).getMillis();
-        <#elseif type == "TimeStamp">
-        out.value = org.joda.time.DateTime.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
-        <#elseif type == "Time">
-        out.value = (int) ((format.parseDateTime(input)).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
-        </#if>
-    }
+    <#if convert.to == "Date" || convert.to == "NullableDate">
+    out.value = (org.joda.time.DateMidnight.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC)).getMillis();
+    <#elseif convert.to == "TimeStamp" || convert.to == "NullableTimeStamp">
+    out.value = org.joda.time.DateTime.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
+    <#elseif convert.to == "Time" || convert.to == "NullableTime">
+    out.value = (int) ((format.parseDateTime(input)).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+    </#if>
+  }
 }
-</#list>
\ No newline at end of file
+</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
index ac1fccba655..3296a0493b6 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -19,11 +19,11 @@
 
 <#list cast.types as type>
 
-<#if type.major == "VarCharDecimalComplex" || type.major == "EmptyStringVarCharDecimalComplex">  <#-- Cast function template for conversion from VarChar to VarDecimal -->
+<#if type.major == "VarCharDecimalComplex" || type.major == "NullableVarCharDecimalComplex">  <#-- Cast function template for conversion from VarChar to VarDecimal -->
 
 <#if type.major == "VarCharDecimalComplex">
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java"/>
-<#elseif type.major == "EmptyStringVarCharDecimalComplex">
+<#elseif type.major == "NullableVarCharDecimalComplex">
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/CastEmptyString${type.from}To${type.to}.java"/>
 </#if>
 
@@ -58,7 +58,7 @@
                   returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
                   nulls = NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
-<#elseif type.major == "EmptyStringVarCharDecimalComplex">
+<#elseif type.major == "NullableVarCharDecimalComplex">
 @FunctionTemplate(name = "castEmptyString${type.from}To${type.to?upper_case}",
                   scope = FunctionTemplate.FunctionScope.SIMPLE,
                   returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
@@ -76,7 +76,7 @@ public void setup() {
   }
 
   public void eval() {
-    <#if type.major == "EmptyStringVarCharDecimalComplex">
+    <#if type.major == "NullableVarCharDecimalComplex">
     // Check if the input is null or empty string
     if (<#if type.from == "NullableVarChar"> in.isSet == 0 || </#if> in.end == in.start) {
       out.isSet = 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index cb0fc5cf2cb..eed4ff830b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -499,8 +499,8 @@ private ExecConstants() {
   public static final PositiveLongValidator SLICE_TARGET_OPTION = new PositiveLongValidator(SLICE_TARGET, Long.MAX_VALUE,
       new OptionDescription("The number of records manipulated within a fragment before Drill parallelizes operations."));
 
-  public static final String CAST_TO_NULLABLE_NUMERIC = "drill.exec.functions.cast_empty_string_to_null";
-  public static final BooleanValidator CAST_TO_NULLABLE_NUMERIC_OPTION = new BooleanValidator(CAST_TO_NULLABLE_NUMERIC,
+  public static final String CAST_EMPTY_STRING_TO_NULL = "drill.exec.functions.cast_empty_string_to_null";
+  public static final BooleanValidator CAST_EMPTY_STRING_TO_NULL_OPTION = new BooleanValidator(CAST_EMPTY_STRING_TO_NULL,
       new OptionDescription("In a text file, treat empty fields as NULL values instead of empty string."));
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index 2a94d9bf7c3..4e35d6dea9a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -60,7 +60,7 @@
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
 import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
-import org.apache.drill.common.expression.fn.CastFunctions;
+import org.apache.drill.common.expression.fn.FunctionReplacementUtils;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.expression.visitors.ConditionalExprOptimizer;
 import org.apache.drill.common.expression.visitors.ExpressionValidator;
@@ -198,7 +198,7 @@ public static LogicalExpression addCastExpression(LogicalExpression fromExpr, Ma
   }
 
   public static LogicalExpression addCastExpression(LogicalExpression fromExpr, MajorType toType, FunctionLookupContext functionLookupContext, ErrorCollector errorCollector, boolean exactResolver) {
-    String castFuncName = CastFunctions.getCastFunc(toType.getMinorType());
+    String castFuncName = FunctionReplacementUtils.getCastFunc(toType.getMinorType());
     List<LogicalExpression> castArgs = Lists.newArrayList();
     castArgs.add(fromExpr);  //input_expr
 
@@ -856,7 +856,7 @@ public LogicalExpression visitCastExpression(CastExpression e, FunctionLookupCon
         MajorType type = e.getMajorType();
 
         // Get the cast function name from the map
-        String castFuncWithType = CastFunctions.getCastFunc(type.getMinorType());
+        String castFuncWithType = FunctionReplacementUtils.getCastFunc(type.getMinorType());
 
         List<LogicalExpression> newArgs = Lists.newArrayList();
         newArgs.add(input);  //input_expr
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
index 5621c44caed..137969a344e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
@@ -33,6 +33,7 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.drill.common.expression.fn.FunctionReplacementUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 import org.apache.drill.shaded.guava.com.google.common.io.Files;
@@ -42,7 +43,6 @@
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.FunctionCall;
-import org.apache.drill.common.expression.fn.CastFunctions;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.RunTimeScan;
 import org.apache.drill.common.scanner.persistence.ScanResult;
@@ -204,16 +204,16 @@ private String functionReplacement(FunctionCall functionCall) {
     if (functionCall.args.size() == 0) {
       return funcName;
     }
-    boolean castToNullableNumeric = optionManager != null &&
-                  optionManager.getOption(ExecConstants.CAST_TO_NULLABLE_NUMERIC_OPTION);
-    if (! castToNullableNumeric) {
+    boolean castEmptyStringToNull = optionManager != null &&
+                  optionManager.getOption(ExecConstants.CAST_EMPTY_STRING_TO_NULL_OPTION);
+    if (!castEmptyStringToNull) {
       return funcName;
     }
     MajorType majorType =  functionCall.args.get(0).getMajorType();
     DataMode dataMode = majorType.getMode();
     MinorType minorType = majorType.getMinorType();
-    if (CastFunctions.isReplacementNeeded(funcName, minorType)) {
-        funcName = CastFunctions.getReplacingCastFunction(funcName, dataMode, minorType);
+    if (FunctionReplacementUtils.isReplacementNeeded(funcName, minorType)) {
+      funcName = FunctionReplacementUtils.getReplacingFunction(funcName, dataMode, minorType);
     }
 
     return funcName;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
index 59236ce2785..8470e1fb033 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -27,6 +27,8 @@
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
 public class MathFunctions{
@@ -70,6 +72,7 @@ public void eval(){
     @Workspace int decimalDigits;
     @Output Float8Holder out;
 
+    @Override
     public void setup() {
       byte[] buf = new byte[right.end - right.start];
       right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
@@ -77,13 +80,14 @@ public void setup() {
       decimalDigits = inputFormat.getMaximumFractionDigits();
     }
 
+    @Override
     public void eval() {
       byte[] buf1 = new byte[left.end - left.start];
       left.buffer.getBytes(left.start, buf1, 0, left.end - left.start);
       String input = new String(buf1);
       try {
         out.value = inputFormat.parse(input).doubleValue();
-      }  catch(java.text.ParseException e) {
+      }  catch (java.text.ParseException e) {
          throw new UnsupportedOperationException("Cannot parse input: " + input + " with pattern : " + inputFormat.toPattern());
       }
 
@@ -93,6 +97,94 @@ public void eval() {
     }
   }
 
+  @FunctionTemplate(name = "convertVarCharToNumber", scope = FunctionScope.SIMPLE, isInternal = true)
+  public static class ToNullableNumber implements DrillSimpleFunc {
+    @Param
+    VarCharHolder left;
+    @Param
+    VarCharHolder right;
+    @Workspace
+    java.text.DecimalFormat inputFormat;
+    @Workspace
+    int decimalDigits;
+    @Output
+    NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      byte[] buf = new byte[right.end - right.start];
+      right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
+      inputFormat = new DecimalFormat(new String(buf));
+      decimalDigits = inputFormat.getMaximumFractionDigits();
+    }
+
+    @Override
+    public void eval() {
+      if (left.start == left.end) {
+        out.isSet = 0;
+        return;
+      }
+      out.isSet = 1;
+
+      byte[] buf1 = new byte[left.end - left.start];
+      left.buffer.getBytes(left.start, buf1, 0, left.end - left.start);
+      String input = new String(buf1);
+      try {
+        out.value = inputFormat.parse(input).doubleValue();
+      } catch (java.text.ParseException e) {
+        throw new UnsupportedOperationException("Cannot parse input: " + input + " with pattern : " + inputFormat.toPattern());
+      }
+
+      // Round the value
+      java.math.BigDecimal roundedValue = new java.math.BigDecimal(out.value);
+      out.value = (roundedValue.setScale(decimalDigits, java.math.BigDecimal.ROUND_HALF_UP)).doubleValue();
+    }
+  }
+
+  @FunctionTemplate(name = "convertNullableVarCharToNumber", scope = FunctionScope.SIMPLE, isInternal = true)
+  public static class ToNullableNumberNullableInput implements DrillSimpleFunc {
+    @Param
+    NullableVarCharHolder left;
+    @Param
+    VarCharHolder right;
+    @Workspace
+    java.text.DecimalFormat inputFormat;
+    @Workspace
+    int decimalDigits;
+    @Output
+    NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      byte[] buf = new byte[right.end - right.start];
+      right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
+      inputFormat = new DecimalFormat(new String(buf));
+      decimalDigits = inputFormat.getMaximumFractionDigits();
+    }
+
+    @Override
+    public void eval() {
+      if (left.isSet == 0 || left.start == left.end) {
+        out.isSet = 0;
+        return;
+      }
+      out.isSet = 1;
+
+      byte[] buf1 = new byte[left.end - left.start];
+      left.buffer.getBytes(left.start, buf1, 0, left.end - left.start);
+      String input = new String(buf1);
+      try {
+        out.value = inputFormat.parse(input).doubleValue();
+      } catch (java.text.ParseException e) {
+        throw new UnsupportedOperationException("Cannot parse input: " + input + " with pattern : " + inputFormat.toPattern());
+      }
+
+      // Round the value
+      java.math.BigDecimal roundedValue = new java.math.BigDecimal(out.value);
+      out.value = (roundedValue.setScale(decimalDigits, java.math.BigDecimal.ROUND_HALF_UP)).doubleValue();
+    }
+  }
+
   @FunctionTemplate(name = "pi", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class Pi implements DrillSimpleFunc {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
index 2b55e3d914a..48a7a90b42b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/stat/RangeExprEvaluator.java
@@ -23,7 +23,7 @@
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.TypedFieldExpr;
 import org.apache.drill.common.expression.ValueExpressions;
-import org.apache.drill.common.expression.fn.CastFunctions;
+import org.apache.drill.common.expression.fn.FunctionReplacementUtils;
 import org.apache.drill.common.expression.fn.FuncHolder;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.types.TypeProtos;
@@ -145,7 +145,7 @@ public long getRowCount() {
 
     final String funcName = ((DrillSimpleFuncHolder) funcHolder).getRegisteredNames()[0];
 
-    if (CastFunctions.isCastFunction(funcName)) {
+    if (FunctionReplacementUtils.isCastFunction(funcName)) {
       Statistics stat = holderExpr.args.get(0).accept(this, null);
       if (stat != null && ! stat.isEmpty()) {
         return evalCastFunc(holderExpr, stat);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index a051d994156..c9d97bf7284 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.impl.project;
 
 import com.carrotsearch.hppc.IntHashSet;
+import org.apache.drill.common.expression.fn.FunctionReplacementUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
@@ -33,7 +34,6 @@
 import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.ValueExpressions;
-import org.apache.drill.common.expression.fn.CastFunctions;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
@@ -604,7 +604,7 @@ private boolean isImplicitFileColumn(ValueVector vvIn) {
       if (Types.isComplex(field.getType()) || Types.isRepeated(field.getType())) {
         final LogicalExpression convertToJson = FunctionCallFactory.createConvert(ConvertExpression.CONVERT_TO, "JSON",
                                                             SchemaPath.getSimplePath(fieldName), ExpressionPosition.UNKNOWN);
-        final String castFuncName = CastFunctions.getCastFunc(MinorType.VARCHAR);
+        final String castFuncName = FunctionReplacementUtils.getCastFunc(MinorType.VARCHAR);
         final List<LogicalExpression> castArgs = Lists.newArrayList();
         castArgs.add(convertToJson);  //input_expr
         // implicitly casting to varchar, since we don't know actual source length, cast to undefined length, which will preserve source length
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
index 8a7439927aa..8991a7d50bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.drill.shaded.guava.com.google.common.base.Function;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import io.netty.buffer.DrillBuf;
@@ -159,7 +160,9 @@ public void reduce(final RexBuilder rexBuilder, List<RexNode> constExps, final L
             .message(message)
             .build(logger);
         }
-        reducedValues.add(rexBuilder.makeNullLiteral(typeFactory.createSqlType(sqlTypeName)));
+
+        RelDataType type = TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, sqlTypeName, true);
+        reducedValues.add(rexBuilder.makeNullLiteral(type));
         continue;
       }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 23f35b5e17b..7badf26b552 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -146,7 +146,7 @@
       new OptionDefinition(ExecConstants.HASHAGG_MIN_BATCHES_PER_PARTITION_VALIDATOR), // for tuning
       new OptionDefinition(ExecConstants.HASHAGG_USE_MEMORY_PREDICTION_VALIDATOR), // for testing
       new OptionDefinition(ExecConstants.HASHAGG_FALLBACK_ENABLED_VALIDATOR), // for enable/disable unbounded HashAgg
-      new OptionDefinition(ExecConstants.CAST_TO_NULLABLE_NUMERIC_OPTION),
+      new OptionDefinition(ExecConstants.CAST_EMPTY_STRING_TO_NULL_OPTION),
       new OptionDefinition(ExecConstants.OUTPUT_FORMAT_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_WRITER_USE_SINGLE_FS_BLOCK_VALIDATOR),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
index be93a56f57f..ad3884920ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -23,7 +23,7 @@
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.TypedFieldExpr;
 import org.apache.drill.common.expression.ValueExpressions;
-import org.apache.drill.common.expression.fn.CastFunctions;
+import org.apache.drill.common.expression.fn.FunctionReplacementUtils;
 import org.apache.drill.common.expression.fn.FuncHolder;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.types.TypeProtos;
@@ -236,7 +236,7 @@ public LogicalExpression visitFunctionHolderExpression(FunctionHolderExpression
       return handleIsFunction(funcHolderExpr, value);
     }
 
-    if (CastFunctions.isCastFunction(funcName)) {
+    if (FunctionReplacementUtils.isCastFunction(funcName)) {
       List<LogicalExpression> newArgs = generateNewExpressions(funcHolderExpr.args, value);
       if (newArgs == null) {
         return null;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastEmptyStrings.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastEmptyStrings.java
index ab47b327722..7f1e4b5d558 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastEmptyStrings.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastEmptyStrings.java
@@ -17,79 +17,134 @@
  */
 package org.apache.drill.exec.fn.impl;
 
-import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.junit.AfterClass;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
-public class TestCastEmptyStrings extends BaseTestQuery {
-  // enable decimal data type
+public class TestCastEmptyStrings extends ClusterTest {
+
   @BeforeClass
-  public static void enableDecimalDataType() throws Exception {
-    test("alter session set `%s` = true", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+  public static void setup() throws Exception {
+    ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
+        // enable decimal data type
+        .sessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true)
+        // Enable the new cast functions (cast empty string "" to null)
+        .systemOption(ExecConstants.CAST_EMPTY_STRING_TO_NULL, true);
+    startCluster(builder);
   }
 
-  @AfterClass
-  public static void disableDecimalDataType() throws Exception {
-    test("alter session set `%s` = false", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+  @Test // see DRILL-1874
+  public void testCastOptionalVarCharToNumeric() throws Exception {
+    testCastOptionalString("columns[0]", "int", "cp.`emptyStrings.csv`", null, 1, 2);
+    testCastOptionalString("columns[0]", "bigint", "cp.`emptyStrings.csv`", null, 1L, 2L);
+    testCastOptionalString("columns[0]", "float", "cp.`emptyStrings.csv`", null, 1.0f, 2.0f);
+    testCastOptionalString("columns[0]", "double", "cp.`emptyStrings.csv`", null, 1.0, 2.0);
   }
 
   @Test // see DRILL-1874
-  public void testCastInputTypeNullableVarCharToNumeric() throws Exception {
-    // Enable the new cast functions (cast empty string "" to null)
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = true;");
-
-    // Test Optional VarChar
-    test("select cast(columns[0] as int) from cp.`emptyStrings.csv`");
-    test("select cast(columns[0] as bigint) from cp.`emptyStrings.csv`");
-    test("select cast(columns[0] as float) from cp.`emptyStrings.csv`");
-    test("select cast(columns[0] as double) from cp.`emptyStrings.csv`");
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = false;");
+  public void testCastRequiredVarCharToNumeric() throws Exception {
+    testCastEmptyString("int");
+    testCastEmptyString("bigint");
+    testCastEmptyString("float");
+    testCastEmptyString("double");
   }
 
   @Test // see DRILL-1874
-  public void testCastInputTypeNonNullableVarCharToNumeric() throws Exception {
-    // Enable the new cast functions (cast empty string "" to null)
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = true;");
-    // Test Required VarChar
-    test("select cast('' as int) from cp.`emptyStrings.csv`");
-    test("select cast('' as bigint) from cp.`emptyStrings.csv`");
-    test("select cast('' as float) from cp.`emptyStrings.csv`");
-    test("select cast('' as double) from cp.`emptyStrings.csv`");
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = false;");
+  public void testCastOptionalVarCharToDecimal() throws Exception {
+    BigDecimal one = BigDecimal.valueOf(1L);
+    BigDecimal two = BigDecimal.valueOf(2L);
+    testCastOptionalString("columns[0]", "decimal", "cp.`emptyStrings.csv`", null, one, two);
+    testCastOptionalString("columns[0]", "decimal(9)", "cp.`emptyStrings.csv`", null, one, two);
+    testCastOptionalString("columns[0]", "decimal(18)", "cp.`emptyStrings.csv`", null, one, two);
+    testCastOptionalString("columns[0]", "decimal(28)", "cp.`emptyStrings.csv`", null, one, two);
+    testCastOptionalString("columns[0]", "decimal(38)", "cp.`emptyStrings.csv`", null, one, two);
   }
 
   @Test // see DRILL-1874
-  public void testCastInputTypeNullableVarCharToDecimal() throws Exception {
-    // Enable the new cast functions (cast empty string "" to null)
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = true;");
+  public void testCastRequiredVarCharToDecimal() throws Exception {
+    testCastEmptyString("decimal");
+    testCastEmptyString("decimal(18)");
+    testCastEmptyString("decimal(28)");
+    testCastEmptyString("decimal(38)");
+  }
 
-    // Test Optional VarChar
-    test("select cast(columns[0] as decimal) from cp.`emptyStrings.csv` where cast(columns[0] as decimal) is null");
-    test("select cast(columns[0] as decimal(9)) from cp.`emptyStrings.csv`");
-    test("select cast(columns[0] as decimal(18)) from cp.`emptyStrings.csv`");
-    test("select cast(columns[0] as decimal(28)) from cp.`emptyStrings.csv`");
-    test("select cast(columns[0] as decimal(38)) from cp.`emptyStrings.csv`");
+  @Test
+  public void testCastRequiredVarCharToDateTime() throws Exception {
+    testCastEmptyString("date");
+    testCastEmptyString("time");
+    testCastEmptyString("timestamp");
+  }
 
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = false;");
+  @Test
+  public void testCastOptionalVarCharToDateTime() throws Exception {
+    testCastOptionalString("dateCol", "date", "cp.`dateWithEmptyStrings.json`",
+        null, null, LocalDate.of(1997, 12, 10));
+    testCastOptionalString("timeCol", "time", "cp.`dateWithEmptyStrings.json`",
+        null, null, LocalTime.of(7, 21, 39));
+    testCastOptionalString("timestampCol", "timestamp", "cp.`dateWithEmptyStrings.json`",
+        null, null, LocalDateTime.of(2003, 9, 11, 10, 1, 37));
   }
 
-  @Test // see DRILL-1874
-  public void testCastInputTypeNonNullableVarCharToDecimal() throws Exception {
-    // Enable the new cast functions (cast empty string "" to null)
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = true;");
+  @Test
+  public void testCastRequiredVarCharToInterval() throws Exception {
+    testCastEmptyString("interval year");
+    testCastEmptyString("interval day");
+    testCastEmptyString("interval month");
+  }
+
+  private void testCastOptionalString(String column, String asType, String table,
+                                      Object... baselineValues) throws Exception {
+    String query = String.format("select cast(%s as %s) c from %s", column, asType, table);
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("c")
+        .baselineValuesForSingleColumn(baselineValues)
+        .go();
+  }
 
-    // Test Required VarChar
-    test("select cast('' as decimal) from cp.`emptyStrings.csv` where cast('' as decimal) is null");
-    test("select cast('' as decimal(18)) from cp.`emptyStrings.csv`");
-    test("select cast('' as decimal(28)) from cp.`emptyStrings.csv`");
-    test("select cast('' as decimal(38)) from cp.`emptyStrings.csv`");
+  private void testCastEmptyString(String asType) throws Exception {
+    Object[] nullObj = new Object[] {null};
+    String query = String.format("select cast('' as %s) c from (values(1))", asType);
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("c")
+        .baselineValues(nullObj)
+        .go();
+  }
+
+  @Test
+  public void testCastOptionalVarCharToNumber() throws Exception {
+    testBuilder()
+        .sqlQuery("select to_number(columns[0], '#,##0.0') n from cp.`emptyStrings.csv`")
+        .unOrdered()
+        .baselineColumns("n")
+        .baselineValuesForSingleColumn(null, 1.0, 2.0)
+        .go();
+  }
 
-    test("alter system set `drill.exec.functions.cast_empty_string_to_null` = false;");
+  @Test
+  public void testCastRequiredVarCharToNumber() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testBuilder()
+        .sqlQuery("select to_number('', '#,##0.0') n from (values(1))")
+        .unOrdered()
+        .baselineColumns("n")
+        .baselineValues(nullObj)
+        .go();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
index 05d383123a9..e0121e6c186 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
@@ -30,12 +30,20 @@
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertThat;
 
 @RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestDateConversions extends BaseTestQuery {
+
+  private static final String ENABLE_CAST_EMPTY_STRING_AS_NULL_QUERY =
+      "alter system set `drill.exec.functions.cast_empty_string_to_null` = true;";
+
   @BeforeClass
   public static void generateTestFiles() throws IOException {
     try (BufferedWriter writer = new BufferedWriter(new FileWriter(new File(dirTestWatcher.getRootDir(), "joda_postgres_date.json")))) {
@@ -206,4 +214,91 @@ public void testPostgresDateFormatError() throws Exception {
       throw e;
     }
   }
+
+  @Test
+  public void testToDateWithEmptyString() throws Exception {
+    String query = "SELECT to_date(dateCol, 'yyyy-MM-dd') d from cp.`dateWithEmptyStrings.json`";
+    testToDateTimeFunctionWithEmptyStringsAsNull(query, "d", null, null, LocalDate.of(1997, 12, 10));
+  }
+
+  @Test
+  public void testToTimeWithEmptyString() throws Exception {
+    String query = "SELECT to_time(timeCol, 'hh:mm:ss') t from cp.`dateWithEmptyStrings.json`";
+    testToDateTimeFunctionWithEmptyStringsAsNull(query, "t", null, null, LocalTime.of(7, 21, 39));
+  }
+
+  @Test
+  public void testToTimeStampWithEmptyString() throws Exception {
+    String query = "SELECT to_timestamp(timestampCol, 'yyyy-MM-dd hh:mm:ss') t from cp.`dateWithEmptyStrings.json`";
+    testToDateTimeFunctionWithEmptyStringsAsNull(query, "t", null, null, LocalDateTime.of(2003, 9, 11, 10, 1, 37));
+  }
+
+  @Test
+  public void testToDateWithLiteralEmptyString() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testToDateTimeFunctionWithEmptyStringsAsNull("SELECT to_date('', 'yyyy-MM-dd') d from (values(1))", "d", nullObj);
+  }
+
+  @Test
+  public void testToTimeWithLiteralEmptyString() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testToDateTimeFunctionWithEmptyStringsAsNull("SELECT to_time('', 'hh:mm:ss') d from (values(1))", "d", nullObj);
+  }
+
+  @Test
+  public void testToTimeStampWithLiteralEmptyString() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testToDateTimeFunctionWithEmptyStringsAsNull(
+        "SELECT to_timestamp('', 'yyyy-MM-dd hh:mm:ss') d from (values(1))", "d", nullObj);
+  }
+
+  @Test
+  public void testSqlToDateWithEmptyString() throws Exception {
+    String query = "SELECT sql_to_date(dateCol, 'yyyy-MM-dd') d from cp.`dateWithEmptyStrings.json`";
+    testToDateTimeFunctionWithEmptyStringsAsNull(query, "d", null, null, LocalDate.of(1997, 12, 10));
+  }
+
+  @Test
+  public void testSqlToTimeWithEmptyString() throws Exception {
+    String query = "SELECT sql_to_time(timeCol, 'HH24:MI:SS') t from cp.`dateWithEmptyStrings.json`";
+    testToDateTimeFunctionWithEmptyStringsAsNull(query, "t", null, null, LocalTime.of(7, 21, 39));
+  }
+
+  @Test
+  public void testSqlToTimeStampWithEmptyString() throws Exception {
+    String query = "SELECT sql_to_timestamp(timestampCol, 'yyyy-MM-dd HH24:MI:SS') t from cp.`dateWithEmptyStrings.json`";
+    testToDateTimeFunctionWithEmptyStringsAsNull(query, "t", null, null, LocalDateTime.of(2003, 9, 11, 10, 1, 37));
+  }
+
+  @Test
+  public void testSqlToDateWithLiteralEmptyString() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testToDateTimeFunctionWithEmptyStringsAsNull(
+        "SELECT sql_to_date('', 'yyyy-MM-dd') d from (values(1))", "d", nullObj);
+  }
+
+  @Test
+  public void testSqlToTimeWithLiteralEmptyString() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testToDateTimeFunctionWithEmptyStringsAsNull(
+        "SELECT sql_to_time('', 'HH24:MI:SS') d from (values(1))", "d", nullObj);
+  }
+
+  @Test
+  public void testSqlToTimeStampWithLiteralEmptyString() throws Exception {
+    Object[] nullObj = new Object[] {null};
+    testToDateTimeFunctionWithEmptyStringsAsNull(
+        "SELECT sql_to_timestamp('', 'yyyy-MM-dd HH24:MI:SS') d from (values(1))", "d", nullObj);
+  }
+
+  private void testToDateTimeFunctionWithEmptyStringsAsNull(
+      String query, String baselineColumn, Object... baselineValues) throws Exception {
+    testBuilder()
+        .optionSettingQueriesForTestQuery(ENABLE_CAST_EMPTY_STRING_AS_NULL_QUERY)
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns(baselineColumn)
+        .baselineValuesForSingleColumn(baselineValues)
+        .go();
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ConfigBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/ConfigBuilder.java
index 4df7af58d1f..9f26b2e007a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ConfigBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ConfigBuilder.java
@@ -142,7 +142,7 @@ public ConfigBuilder put(String key, Object value) {
   private static Properties createDefaultProperties()
   {
     Properties properties = new Properties();
-    properties.put(ExecConstants.CAST_TO_NULLABLE_NUMERIC, "false");
+    properties.put(ExecConstants.CAST_EMPTY_STRING_TO_NULL, "false");
     properties.put(ExecConstants.USE_DYNAMIC_UDFS_KEY, "false");
     properties.put(ExecConstants.SYS_STORE_PROVIDER_LOCAL_ENABLE_WRITE, "false");
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
index 775f5464c55..d45bd6f2e87 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
@@ -25,6 +25,7 @@
 import java.time.OffsetDateTime;
 import java.time.ZoneOffset;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -353,6 +354,24 @@ public TestBuilder baselineValues(Object... baselineValues) {
     return this;
   }
 
+  /**
+   * This method is used to pass in an array of values for records verification in case if
+   * {@link #baselineColumns(String...)} specifies one column only without
+   * the need to create a CSV or JSON file to store the baseline.
+   *
+   * This can be called repeatedly to pass an array of records to verify. It works for both ordered and unordered
+   * checks.
+   *
+   * @param baselineValues baseline values for a single column to validate
+   * @return {@code this} test builder
+   */
+  public TestBuilder baselineValuesForSingleColumn(Object... baselineValues) {
+    assertEquals("Only one column should be specified", 1, baselineColumns.length);
+    Arrays.stream(baselineValues)
+        .forEach(this::baselineValues);
+    return this;
+  }
+
   /**
    * This can be used in cases where we want to avoid issues with the assumptions made by the test framework.
    * Most of the methods for verification in the framework run drill queries to generate the read baseline files or
diff --git a/exec/java-exec/src/test/resources/dateWithEmptyStrings.json b/exec/java-exec/src/test/resources/dateWithEmptyStrings.json
new file mode 100644
index 00000000000..1762ee57b49
--- /dev/null
+++ b/exec/java-exec/src/test/resources/dateWithEmptyStrings.json
@@ -0,0 +1,3 @@
+{"dateCol": null, "timeCol": "", "timestampCol": "2003-09-11 10:01:37"}
+{"dateCol": "1997-12-10", "timeCol": null, "timestampCol": ""}
+{"dateCol": "", "timeCol": "07:21:39", "timestampCol": null}
diff --git a/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java b/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
deleted file mode 100644
index b5ed5b6a2f3..00000000000
--- a/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * 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.drill.common.expression.fn;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public class CastFunctions {
-
-  private static Map<MinorType, String> TYPE2FUNC = new HashMap<>();
-  /** The cast functions that need to be replaced (if
-   * "drill.exec.functions.cast_empty_string_to_null" is set to true). */
-  private static Set<String> CAST_FUNC_REPLACEMENT_NEEDED = new HashSet<>();
-  /** Map from the replaced functions to the new ones (for non-nullable VARCHAR). */
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR = new HashMap<>();
-  /** Map from the replaced functions to the new ones (for non-nullable VAR16CHAR). */
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR = new HashMap<>();
-  /** Map from the replaced functions to the new ones (for non-nullable VARBINARY). */
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY = new HashMap<>();
-  /** Map from the replaced functions to the new ones (for nullable VARCHAR). */
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR = new HashMap<>();
-  /** Map from the replaced functions to the new ones (for nullable VAR16CHAR). */
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR = new HashMap<>();
-  /** Map from the replaced functions to the new ones (for nullable VARBINARY). */
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY = new HashMap<>();
-  static {
-    TYPE2FUNC.put(MinorType.UNION, "castUNION");
-    TYPE2FUNC.put(MinorType.BIGINT, "castBIGINT");
-    TYPE2FUNC.put(MinorType.INT, "castINT");
-    TYPE2FUNC.put(MinorType.BIT, "castBIT");
-    TYPE2FUNC.put(MinorType.TINYINT, "castTINYINT");
-    TYPE2FUNC.put(MinorType.FLOAT4, "castFLOAT4");
-    TYPE2FUNC.put(MinorType.FLOAT8, "castFLOAT8");
-    TYPE2FUNC.put(MinorType.VARCHAR, "castVARCHAR");
-    TYPE2FUNC.put(MinorType.VAR16CHAR, "castVAR16CHAR");
-    TYPE2FUNC.put(MinorType.VARBINARY, "castVARBINARY");
-    TYPE2FUNC.put(MinorType.DATE, "castDATE");
-    TYPE2FUNC.put(MinorType.TIME, "castTIME");
-    TYPE2FUNC.put(MinorType.TIMESTAMP, "castTIMESTAMP");
-    TYPE2FUNC.put(MinorType.TIMESTAMPTZ, "castTIMESTAMPTZ");
-    TYPE2FUNC.put(MinorType.INTERVALDAY, "castINTERVALDAY");
-    TYPE2FUNC.put(MinorType.INTERVALYEAR, "castINTERVALYEAR");
-    TYPE2FUNC.put(MinorType.INTERVAL, "castINTERVAL");
-    TYPE2FUNC.put(MinorType.DECIMAL9, "castDECIMAL9");
-    TYPE2FUNC.put(MinorType.DECIMAL18, "castDECIMAL18");
-    TYPE2FUNC.put(MinorType.DECIMAL28SPARSE, "castDECIMAL28SPARSE");
-    TYPE2FUNC.put(MinorType.DECIMAL28DENSE, "castDECIMAL28DENSE");
-    TYPE2FUNC.put(MinorType.DECIMAL38SPARSE, "castDECIMAL38SPARSE");
-    TYPE2FUNC.put(MinorType.DECIMAL38DENSE, "castDECIMAL38DENSE");
-    TYPE2FUNC.put(MinorType.VARDECIMAL, "castVARDECIMAL");
-
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.INT));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.BIGINT));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.FLOAT4));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.FLOAT8));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL9));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL18));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE));
-    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.VARDECIMAL));
-
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringVarCharToNullableINT");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringVarCharToNullableBIGINT");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.FLOAT4), "castEmptyStringVarCharToNullableFLOAT4");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.FLOAT8), "castEmptyStringVarCharToNullableFLOAT8");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL9), "castEmptyStringVarCharToNullableDECIMAL9");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringVarCharToNullableDECIMAL18");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringVarCharToNullableDECIMAL28SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringVarCharToNullableDECIMAL38SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringVarCharToNullableVARDECIMAL");
-
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringVar16CharToNullableINT");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringVar16CharToNullableBIGINT");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.FLOAT4), "castEmptyStringVar16CharToNullableFLOAT4");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.FLOAT8), "castEmptyStringVar16CharToNullableFLOAT8");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL9), "castEmptyStringVar16CharToNullableDECIMAL9");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringVar16CharToNullableDECIMAL18");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringVar16CharToNullableDECIMAL28SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringVar16CharToNullableDECIMAL38SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringVar16CharToNullableVARDECIMAL");
-
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringVarBinaryToNullableINT");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringVarBinaryToNullableBIGINT");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.FLOAT4), "castEmptyStringVarBinaryToNullableFLOAT4");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.FLOAT8), "castEmptyStringVarBinaryToNullableFLOAT8");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL9), "castEmptyStringVarBinaryToNullableDECIMAL9");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringVarBinaryToNullableDECIMAL18");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringVarBinaryToNullableDECIMAL28SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringVarBinaryToNullableDECIMAL38SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringVarBinaryToNullableVARDECIMAL");
-
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringNullableVarCharToNullableINT");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringNullableVarCharToNullableBIGINT");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.FLOAT4), "castEmptyStringNullableVarCharToNullableFLOAT4");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.FLOAT8), "castEmptyStringNullableVarCharToNullableFLOAT8");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL9), "castEmptyStringNullableVarCharToNullableDECIMAL9");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringNullableVarCharToNullableDECIMAL18");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringNullableVarCharToNullableDECIMAL28SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringNullableVarCharToNullableDECIMAL38SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringNullableVarCharToNullableVARDECIMAL");
-
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringNullableVar16CharToNullableINT");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringNullableVar16CharToNullableBIGINT");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.FLOAT4), "castEmptyStringNullableVar16CharToNullableFLOAT4");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.FLOAT8), "castEmptyStringNullableVar16CharToNullableFLOAT8");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL9), "castEmptyStringNullableVar16CharToNullableDECIMAL9");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringNullableVar16CharToNullableDECIMAL18");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringNullableVar16CharToNullableDECIMAL28SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringNullableVar16CharToNullableDECIMAL38SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringNullableVar16CharToNullableVARDECIMAL");
-
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringNullableVarBinaryToNullableINT");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringNullableVarBinaryToNullableBIGINT");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.FLOAT4), "castEmptyStringNullableVarBinaryToNullableFLOAT4");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.FLOAT8), "castEmptyStringNullableVarBinaryToNullableFLOAT8");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL9), "castEmptyStringNullableVarBinaryToNullableDECIMAL9");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringNullableVarBinaryToNullableDECIMAL18");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringNullableVarBinaryToNullableDECIMAL28SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringNullableVarBinaryToNullableDECIMAL38SPARSE");
-    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringNullableVarBinaryToNullableVARDECIMAL");
-  }
-
-  /**
-  * Given the target type, get the appropriate cast function
-  * @param targetMinorType the target data type
-  * @return the name of cast function
-  */
-  public static String getCastFunc(MinorType targetMinorType) {
-    String func = TYPE2FUNC.get(targetMinorType);
-    if (func != null) {
-      return func;
-    }
-
-    throw new IllegalArgumentException(
-      String.format("cast function for type %s is not defined", targetMinorType.name()));
-  }
-
-  /**
-  * Get a replacing cast function for the original function, based on the specified data mode
-  * @param originalCastFunction original cast function
-  * @param dataMode data mode of the input data
-  * @param inputType input (minor) type for cast
-  * @return the name of replaced cast function
-  */
-  public static String getReplacingCastFunction(String originalCastFunction, DataMode dataMode, MinorType inputType) {
-    if(dataMode == DataMode.OPTIONAL) {
-      return getReplacingCastFunctionFromNullable(originalCastFunction, inputType);
-    }
-
-    if(dataMode == DataMode.REQUIRED) {
-      return getReplacingCastFunctionFromNonNullable(originalCastFunction, inputType);
-    }
-
-    throw new RuntimeException(
-       String.format("replacing cast function for datatype %s is not defined", dataMode));
-  }
-
-  /**
-  * Check if a replacing cast function is available for the the original function
-  * @param originalfunction original cast function
-  * @param inputType input (minor) type for cast
-  * @return true if replacement is needed, false - if isn't
-  */
-  public static boolean isReplacementNeeded(String originalfunction, MinorType inputType) {
-    return (inputType == MinorType.VARCHAR || inputType == MinorType.VARBINARY || inputType == MinorType.VAR16CHAR) &&
-        CAST_FUNC_REPLACEMENT_NEEDED.contains(originalfunction);
-  }
-
-  /**
-   * Check if a funcName is one of the cast function.
-   * @param funcName
-   * @return
-   */
-  public static boolean isCastFunction(String funcName) {
-    return TYPE2FUNC.values().contains(funcName);
-  }
-
-  private static String getReplacingCastFunctionFromNonNullable(String originalCastFunction, MinorType inputType) {
-    if(inputType == MinorType.VARCHAR && CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.containsKey(originalCastFunction)) {
-      return CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.get(originalCastFunction);
-    }
-    if(inputType == MinorType.VAR16CHAR && CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.containsKey(originalCastFunction)) {
-      return CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.get(originalCastFunction);
-    }
-    if(inputType == MinorType.VARBINARY && CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.containsKey(originalCastFunction)) {
-      return CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.get(originalCastFunction);
-    }
-
-    throw new RuntimeException(
-      String.format("replacing cast function for %s is not defined", originalCastFunction));
-  }
-
-  private static String getReplacingCastFunctionFromNullable(String originalCastFunction, MinorType inputType) {
-    if(inputType == MinorType.VARCHAR && CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.containsKey(originalCastFunction)) {
-      return CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.get(originalCastFunction);
-    }
-    if(inputType == MinorType.VAR16CHAR && CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.containsKey(originalCastFunction)) {
-      return CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.get(originalCastFunction);
-    }
-    if(inputType == MinorType.VARBINARY && CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.containsKey(originalCastFunction)) {
-      return CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.get(originalCastFunction);
-    }
-
-    throw new RuntimeException(
-      String.format("replacing cast function for %s is not defined", originalCastFunction));
-  }
-}
diff --git a/logical/src/main/java/org/apache/drill/common/expression/fn/FunctionReplacementUtils.java b/logical/src/main/java/org/apache/drill/common/expression/fn/FunctionReplacementUtils.java
new file mode 100644
index 00000000000..982423ee4f6
--- /dev/null
+++ b/logical/src/main/java/org/apache/drill/common/expression/fn/FunctionReplacementUtils.java
@@ -0,0 +1,233 @@
+/*
+ * 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.drill.common.expression.fn;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+public class FunctionReplacementUtils {
+
+  private static final Map<MinorType, String> TYPE_TO_CAST_FUNC = new HashMap<>();
+  // Maps function to supported input types for substitution
+  private static final Map<String, Set<MinorType>> FUNC_TO_INPUT_TYPES = new HashMap<>();
+  /**
+   * The functions that need to be replaced (if
+   * {@code "drill.exec.functions.cast_empty_string_to_null"} is set to {@code true}).
+   */
+  private static final Set<String> FUNC_REPLACEMENT_NEEDED = new HashSet<>();
+  /** Map from the replaced functions to the new ones (for non-nullable VARCHAR). */
+  private static final Map<String, String> FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARCHAR = new HashMap<>();
+  /** Map from the replaced functions to the new ones (for non-nullable VAR16CHAR). */
+  private static final Map<String, String> FUNC_REPLACEMENT_FROM_NON_NULLABLE_VAR16CHAR = new HashMap<>();
+  /** Map from the replaced functions to the new ones (for non-nullable VARBINARY). */
+  private static final Map<String, String> FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARBINARY = new HashMap<>();
+  /** Map from the replaced functions to the new ones (for nullable VARCHAR). */
+  private static final Map<String, String> FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR = new HashMap<>();
+  /** Map from the replaced functions to the new ones (for nullable VAR16CHAR). */
+  private static final Map<String, String> FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR = new HashMap<>();
+  /** Map from the replaced functions to the new ones (for nullable VARBINARY). */
+  private static final Map<String, String> FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY = new HashMap<>();
+
+  static {
+    initCastFunctionSubstitutions();
+    initToFunctionSubstitutions();
+  }
+
+  private static void initCastFunctionSubstitutions() {
+    TYPE_TO_CAST_FUNC.put(MinorType.UNION, "castUNION");
+    TYPE_TO_CAST_FUNC.put(MinorType.BIGINT, "castBIGINT");
+    TYPE_TO_CAST_FUNC.put(MinorType.INT, "castINT");
+    TYPE_TO_CAST_FUNC.put(MinorType.BIT, "castBIT");
+    TYPE_TO_CAST_FUNC.put(MinorType.TINYINT, "castTINYINT");
+    TYPE_TO_CAST_FUNC.put(MinorType.FLOAT4, "castFLOAT4");
+    TYPE_TO_CAST_FUNC.put(MinorType.FLOAT8, "castFLOAT8");
+    TYPE_TO_CAST_FUNC.put(MinorType.VARCHAR, "castVARCHAR");
+    TYPE_TO_CAST_FUNC.put(MinorType.VAR16CHAR, "castVAR16CHAR");
+    TYPE_TO_CAST_FUNC.put(MinorType.VARBINARY, "castVARBINARY");
+    TYPE_TO_CAST_FUNC.put(MinorType.DATE, "castDATE");
+    TYPE_TO_CAST_FUNC.put(MinorType.TIME, "castTIME");
+    TYPE_TO_CAST_FUNC.put(MinorType.TIMESTAMP, "castTIMESTAMP");
+    TYPE_TO_CAST_FUNC.put(MinorType.TIMESTAMPTZ, "castTIMESTAMPTZ");
+    TYPE_TO_CAST_FUNC.put(MinorType.INTERVALDAY, "castINTERVALDAY");
+    TYPE_TO_CAST_FUNC.put(MinorType.INTERVALYEAR, "castINTERVALYEAR");
+    TYPE_TO_CAST_FUNC.put(MinorType.INTERVAL, "castINTERVAL");
+    TYPE_TO_CAST_FUNC.put(MinorType.DECIMAL9, "castDECIMAL9");
+    TYPE_TO_CAST_FUNC.put(MinorType.DECIMAL18, "castDECIMAL18");
+    TYPE_TO_CAST_FUNC.put(MinorType.DECIMAL28SPARSE, "castDECIMAL28SPARSE");
+    TYPE_TO_CAST_FUNC.put(MinorType.DECIMAL28DENSE, "castDECIMAL28DENSE");
+    TYPE_TO_CAST_FUNC.put(MinorType.DECIMAL38SPARSE, "castDECIMAL38SPARSE");
+    TYPE_TO_CAST_FUNC.put(MinorType.DECIMAL38DENSE, "castDECIMAL38DENSE");
+    TYPE_TO_CAST_FUNC.put(MinorType.VARDECIMAL, "castVARDECIMAL");
+
+    // Numeric types
+    setupReplacementFunctionsForCast(MinorType.INT, "NullableINT");
+    setupReplacementFunctionsForCast(MinorType.BIGINT, "NullableBIGINT");
+    setupReplacementFunctionsForCast(MinorType.FLOAT4, "NullableFLOAT4");
+    setupReplacementFunctionsForCast(MinorType.FLOAT8, "NullableFLOAT8");
+    setupReplacementFunctionsForCast(MinorType.DECIMAL9, "NullableDECIMAL9");
+    setupReplacementFunctionsForCast(MinorType.DECIMAL18, "NullableDECIMAL18");
+    setupReplacementFunctionsForCast(MinorType.DECIMAL28SPARSE, "NullableDECIMAL28SPARSE");
+    setupReplacementFunctionsForCast(MinorType.DECIMAL38SPARSE, "NullableDECIMAL38SPARSE");
+    setupReplacementFunctionsForCast(MinorType.VARDECIMAL, "NullableVARDECIMAL");
+    // date/time types
+    setupReplacementFunctionsForCast(MinorType.DATE, "NULLABLEDATE");
+    setupReplacementFunctionsForCast(MinorType.TIME, "NULLABLETIME");
+    setupReplacementFunctionsForCast(MinorType.TIMESTAMP, "NULLABLETIMESTAMP");
+    // interval types
+    setupReplacementFunctionsForCast(MinorType.INTERVAL, "NullableINTERVAL");
+    setupReplacementFunctionsForCast(MinorType.INTERVALDAY, "NullableINTERVALDAY");
+    setupReplacementFunctionsForCast(MinorType.INTERVALYEAR, "NullableINTERVALYEAR");
+  }
+
+  private static void initToFunctionSubstitutions() {
+    setupReplacementFunctionsForTo("to_number", "ToNumber");
+
+    setupReplacementFunctionsForTo("to_date", "ToNullableDate");
+    setupReplacementFunctionsForTo("to_time", "ToNullableTime");
+    setupReplacementFunctionsForTo("to_timestamp", "ToNullableTimeStamp");
+
+    setupReplacementFunctionsForTo("sql_to_date", "SqlToNullableDate");
+    setupReplacementFunctionsForTo("sql_to_time", "SqlToNullableTime");
+    setupReplacementFunctionsForTo("sql_to_timestamp", "SqlToNullableTimeStamp");
+  }
+
+  private static void setupReplacementFunctionsForCast(MinorType type, String toSuffix) {
+    String functionName = TYPE_TO_CAST_FUNC.get(type);
+
+    FUNC_REPLACEMENT_NEEDED.add(functionName);
+    Set<MinorType> supportedInputTypes = new HashSet<>(
+        Arrays.asList(MinorType.VARCHAR, MinorType.VAR16CHAR, MinorType.VARBINARY));
+    FUNC_TO_INPUT_TYPES.put(functionName, supportedInputTypes);
+
+    FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARCHAR.put(functionName, "castEmptyStringVarCharTo" + toSuffix);
+    FUNC_REPLACEMENT_FROM_NON_NULLABLE_VAR16CHAR.put(functionName, "castEmptyStringVar16CharTo" + toSuffix);
+    FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARBINARY.put(functionName, "castEmptyStringVarBinaryTo" + toSuffix);
+
+    FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(functionName, "castEmptyStringNullableVarCharTo" + toSuffix);
+    FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(functionName, "castEmptyStringNullableVar16CharTo" + toSuffix);
+    FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(functionName, "castEmptyStringNullableVarBinaryTo" + toSuffix);
+  }
+
+  private static void setupReplacementFunctionsForTo(String functionName, String toSuffix) {
+    Set<MinorType> typeSet = Collections.singleton(MinorType.VARCHAR);
+    FUNC_TO_INPUT_TYPES.put(functionName, typeSet);
+    FUNC_REPLACEMENT_NEEDED.add(functionName);
+
+    FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARCHAR.put(functionName,"convertVarChar" + toSuffix);
+    FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(functionName, "convertNullableVarChar" + toSuffix);
+  }
+
+  /**
+  * Given the target type, get the appropriate cast function
+  * @param targetMinorType the target data type
+  * @return the name of cast function
+  */
+  public static String getCastFunc(MinorType targetMinorType) {
+    String func = TYPE_TO_CAST_FUNC.get(targetMinorType);
+    if (func != null) {
+      return func;
+    }
+
+    throw new IllegalArgumentException(
+      String.format("cast function for type %s is not defined", targetMinorType.name()));
+  }
+
+  /**
+  * Get a replacing function for the original function, based on the specified data mode
+  * @param functionName original function name
+  * @param dataMode data mode of the input data
+  * @param inputType input (minor) type
+  * @return the name of replaced function
+  */
+  public static String getReplacingFunction(String functionName, DataMode dataMode, MinorType inputType) {
+    if (dataMode == DataMode.OPTIONAL) {
+      return getReplacingFunctionFromNullable(functionName, inputType);
+    }
+
+    if (dataMode == DataMode.REQUIRED) {
+      return getReplacingFunctionFromNonNullable(functionName, inputType);
+    }
+
+    throw new DrillRuntimeException(
+       String.format("replacing function '%s' for datatype %s is not defined", functionName, dataMode));
+  }
+
+  /**
+  * Check if a replacing function is available for the the original function
+  * @param functionName original function name
+  * @param inputType input (minor) type
+  * @return {@code true} if replacement is needed, {@code false} otherwise
+  */
+  public static boolean isReplacementNeeded(String functionName, MinorType inputType) {
+    return FUNC_REPLACEMENT_NEEDED.contains(functionName)
+        && FUNC_TO_INPUT_TYPES.get(functionName).contains(inputType);
+  }
+
+  /**
+   * Check if a function is a cast function.
+   * @param functionName name of the function
+   * @return {@code true} if function is CAST function, {@code false} otherwise
+   */
+  public static boolean isCastFunction(String functionName) {
+    return TYPE_TO_CAST_FUNC.values().contains(functionName);
+  }
+
+  private static String getReplacingFunctionFromNonNullable(String functionName, MinorType inputType) {
+    if (inputType == MinorType.VARCHAR
+        && FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARCHAR.containsKey(functionName)) {
+      return FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARCHAR.get(functionName);
+    }
+    if (inputType == MinorType.VAR16CHAR
+        && FUNC_REPLACEMENT_FROM_NON_NULLABLE_VAR16CHAR.containsKey(functionName)) {
+      return FUNC_REPLACEMENT_FROM_NON_NULLABLE_VAR16CHAR.get(functionName);
+    }
+    if (inputType == MinorType.VARBINARY
+        && FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARBINARY.containsKey(functionName)) {
+      return FUNC_REPLACEMENT_FROM_NON_NULLABLE_VARBINARY.get(functionName);
+    }
+
+    throw new DrillRuntimeException(
+      String.format("replacing function for %s is not defined", functionName));
+  }
+
+  private static String getReplacingFunctionFromNullable(String functionName, MinorType inputType) {
+    if (inputType == MinorType.VARCHAR
+        && FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.containsKey(functionName)) {
+      return FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.get(functionName);
+    }
+    if (inputType == MinorType.VAR16CHAR
+        && FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.containsKey(functionName)) {
+      return FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.get(functionName);
+    }
+    if (inputType == MinorType.VARBINARY
+        && FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.containsKey(functionName)) {
+      return FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.get(functionName);
+    }
+
+    throw new DrillRuntimeException(
+      String.format("replacing function for %s is not defined", functionName));
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Improve to_date, to_time and to_timestamp and corresponding cast functions to handle empty string when `drill.exec.functions.cast_empty_string_to_null` option is enabled
> -------------------------------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: DRILL-6768
>                 URL: https://issues.apache.org/jira/browse/DRILL-6768
>             Project: Apache Drill
>          Issue Type: Improvement
>    Affects Versions: 1.14.0
>            Reporter: Bohdan Kazydub
>            Assignee: Bohdan Kazydub
>            Priority: Major
>              Labels: doc-impacting, ready-to-commit
>             Fix For: 1.15.0
>
>
> When `drill.exec.functions.cast_empty_string_to_null` option is enabled
> `to_date`, `to_time` and `to_timestamp` functions while converting string to according type in case if null or empty string values are passed will return NULL (to avoid CASE clauses which are littering a query and will work in accordance with their respective CAST counterparts) for both cases.
>  
>   
>   
> CASTs will  be handled in a similar way (uniformly with numeric types):
>  
> ||Value to cast||Now||Will be||
> |NULL|NULL|NULL|
> |'' (empty string)|Error in many cases (except numerical types)|NULL|
>  CAST empty string to null (in case of enabled option) will be supported by DATE, TIME, TIMESTAMP, INTERVAL YEAR, INTERVAL MONTH and INTERVAL DAY functions in addition to numeric types.
>  
> *For documentation*
> TBA



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)