You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2018/05/12 00:06:35 UTC

[drill] branch master updated: DRILL-6242 Use java.time.Local{Date|Time|DateTime} for Drill Date, Time, Timestamp types. (#3)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c1f0adc  DRILL-6242 Use java.time.Local{Date|Time|DateTime} for Drill Date, Time, Timestamp types. (#3)
c1f0adc is described below

commit c1f0adc9276ab8314ce9c5dff2d9c92066c71530
Author: jiang-wu <jw...@alumni.stanford.edu>
AuthorDate: Tue May 1 14:48:26 2018 -0700

    DRILL-6242 Use java.time.Local{Date|Time|DateTime} for Drill Date, Time, Timestamp types. (#3)
    
    close apache/drill#1247
    
    * DRILL-6242 - Use java.time.Local{Date|Time|DateTime} classes to hold values from corresponding Drill date, time, and timestamp types.
    Conflicts:
    	exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
    
    Fix merge conflicts and check style.
---
 .../exec/TestHiveDrillNativeParquetReader.java     |  20 +-
 .../drill/exec/fn/hive/TestInbuiltHiveUDFs.java    |  15 +-
 .../apache/drill/exec/hive/TestHiveStorage.java    |  37 ++--
 .../exec/store/jdbc/TestJdbcPluginWithDerbyIT.java |  12 +-
 .../exec/store/jdbc/TestJdbcPluginWithMySQLIT.java |  15 +-
 .../main/codegen/templates/CastDateVarChar.java    |   8 +-
 .../main/codegen/templates/CastVarCharDate.java    |   8 +-
 .../drill/exec/expr/fn/impl/DateTypeFunctions.java |   8 +-
 .../exec/planner/common/DrillValuesRelBase.java    |  30 +--
 .../drill/exec/store/bson/BsonRecordReader.java    |  25 ++-
 .../org/apache/drill/exec/util/VectorUtil.java     |  10 +-
 .../exec/vector/complex/fn/BasicJsonOutput.java    |  36 ++--
 .../exec/vector/complex/fn/ExtendedJsonOutput.java |  15 +-
 .../drill/exec/vector/complex/fn/JsonOutput.java   |   8 +-
 .../drill/exec/vector/complex/fn/VectorOutput.java |  34 +--
 .../java/org/apache/drill/TestFunctionsQuery.java  |  22 +-
 .../java/org/apache/drill/TestStarQueries.java     |   2 +-
 .../drill/exec/fn/impl/TestCastFunctions.java      |  27 +--
 .../drill/exec/fn/impl/TestDateAddFunctions.java   |  29 +--
 .../drill/exec/fn/impl/TestDateTruncFunctions.java | 139 ++++++------
 .../drill/exec/fn/impl/TestNewDateFunctions.java   |  10 +-
 .../exec/fn/interp/ExpressionInterpreterTest.java  |   8 +-
 .../exec/physical/impl/TestConvertFunctions.java   |  13 +-
 .../physical/impl/TestNestedDateTimeTimestamp.java | 234 +++++++++++++++++++++
 .../impl/limit/TestEarlyLimit0Optimization.java    |  16 +-
 .../writer/TestCorruptParquetDateCorrection.java   |  66 +++---
 .../physical/impl/writer/TestParquetWriter.java    |  32 +--
 .../planner/logical/TestCaseNullableTypes.java     |   5 +-
 .../drill/exec/store/avro/AvroFormatTest.java      |  75 ++++---
 .../exec/store/bson/TestBsonRecordReader.java      |   5 +-
 .../exec/store/parquet/TestParquetComplex.java     |  52 ++---
 .../store/parquet2/TestDrillParquetReader.java     |  51 ++---
 .../vector/complex/writer/TestExtendedTypes.java   |   2 +-
 .../java/org/apache/drill/test/TestBuilder.java    |  19 +-
 .../test/resources/baseline_nested_datetime.json   |  13 ++
 exec/java-exec/src/test/resources/datetime.parquet | Bin 0 -> 1217 bytes
 .../src/main/codegen/data/ValueVectorTypes.tdd     |   6 +-
 .../src/main/codegen/includes/vv_imports.ftl       |   8 +-
 .../codegen/templates/AbstractFieldReader.java     |   2 +-
 .../main/codegen/templates/FixedValueVectors.java  |  15 +-
 .../main/codegen/templates/HolderReaderImpl.java   |  12 ++
 .../src/main/codegen/templates/NullReader.java     |   2 +-
 .../src/main/codegen/templates/UnionReader.java    |   2 +-
 .../drill/exec/expr/fn/impl/DateUtility.java       | 107 ++++++++--
 .../drill/exec/util/JsonStringArrayList.java       |   1 +
 .../apache/drill/exec/util/JsonStringHashMap.java  |   1 +
 .../drill/exec/util/SerializationModule.java       |  75 +++++++
 47 files changed, 895 insertions(+), 437 deletions(-)

diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java
index 23c67b5..fd9701c 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java
@@ -17,14 +17,19 @@
  */
 package org.apache.drill.exec;
 
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
+
+import java.math.BigDecimal;
+
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.exceptions.UserRemoteException;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.hive.HiveTestBase;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.hamcrest.CoreMatchers;
-import org.joda.time.DateTime;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -32,13 +37,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Timestamp;
-
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.assertEquals;
-
 @Category({SlowTest.class, HiveStorageTest.class})
 public class TestHiveDrillNativeParquetReader extends HiveTestBase {
 
@@ -227,14 +225,14 @@ public class TestHiveDrillNativeParquetReader extends HiveTestBase {
         // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now.
         //"binary_part",
         "boolean_part", "tinyint_part", "decimal0_part", "decimal9_part", "decimal18_part", "decimal28_part", "decimal38_part", "double_part", "float_part", "int_part", "bigint_part", "smallint_part", "string_part", "varchar_part", "timestamp_part", "date_part", "char_part")
-        .baselineValues("binaryfield".getBytes(), false, 34, new BigDecimal("66"), new BigDecimal("2347.92"), new BigDecimal("2758725827.99990"), new BigDecimal("29375892739852.8"), new BigDecimal("89853749534593985.783"), 8.345d, 4.67f, 123456, 234235L, 3455, "stringfield", "varcharfield", new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), "charfield",
+        .baselineValues("binaryfield".getBytes(), false, 34, new BigDecimal("66"), new BigDecimal("2347.92"), new BigDecimal("2758725827.99990"), new BigDecimal("29375892739852.8"), new BigDecimal("89853749534593985.783"), 8.345d, 4.67f, 123456, 234235L, 3455, "stringfield", "varcharfield", DateUtility.parseBest("2013-07-05 17:01:00"), "charfield",
         // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now.
         //"binary",
-        true, 64, new BigDecimal("37"), new BigDecimal("36.90"), new BigDecimal("3289379872.94565"), new BigDecimal("39579334534534.4"), new BigDecimal("363945093845093890.900"), 8.345d, 4.67f, 123456, 234235L, 3455, "string", "varchar", new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), new DateTime(Date.valueOf("2013-07-05").getTime()), "char").baselineValues( // All fields are null, but partition fields have non-null values
+        true, 64, new BigDecimal("37"), new BigDecimal("36.90"), new BigDecimal("3289379872.94565"), new BigDecimal("39579334534534.4"), new BigDecimal("363945093845093890.900"), 8.345d, 4.67f, 123456, 234235L, 3455, "string", "varchar", DateUtility.parseBest("2013-07-05 17:01:00"), DateUtility.parseLocalDate("2013-07-05"), "char").baselineValues( // All fields are null, but partition fields have non-null values
         null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
         // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now.
         //"binary",
-        true, 64, new BigDecimal("37"), new BigDecimal("36.90"), new BigDecimal("3289379872.94565"), new BigDecimal("39579334534534.4"), new BigDecimal("363945093845093890.900"), 8.345d, 4.67f, 123456, 234235L, 3455, "string", "varchar", new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), new DateTime(Date.valueOf("2013-07-05").getTime()), "char").go();
+        true, 64, new BigDecimal("37"), new BigDecimal("36.90"), new BigDecimal("3289379872.94565"), new BigDecimal("39579334534534.4"), new BigDecimal("363945093845093890.900"), 8.345d, 4.67f, 123456, 234235L, 3455, "string", "varchar", DateUtility.parseBest("2013-07-05 17:01:00"), DateUtility.parseLocalDate("2013-07-05"), "char").go();
   }
 
   @Test // DRILL-3938
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
index d4e0b5c..95acbd7 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
@@ -17,22 +17,23 @@
  */
 package org.apache.drill.exec.fn.hive;
 
-import com.google.common.collect.Lists;
+import java.time.LocalDateTime;
+import java.util.List;
+
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.categories.HiveStorageTest;
-import org.apache.drill.test.QueryTestUtil;
-import org.apache.drill.test.TestBuilder;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.hive.HiveTestBase;
 import org.apache.drill.exec.server.options.OptionValue;
-import org.joda.time.DateTime;
+import org.apache.drill.test.QueryTestUtil;
+import org.apache.drill.test.TestBuilder;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.List;
+import com.google.common.collect.Lists;
 
 @Category({SlowTest.class, HiveStorageTest.class})
 public class TestInbuiltHiveUDFs extends HiveTestBase {
@@ -169,7 +170,7 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
         .sqlQuery("select from_utc_timestamp('1970-01-01 08:00:00','PST') as PST_TIMESTAMP from (VALUES(1))")
         .unOrdered()
         .baselineColumns("PST_TIMESTAMP")
-        .baselineValues(DateTime.parse("1970-01-01T00:00:00.0"))
+        .baselineValues(LocalDateTime.parse("1970-01-01T00:00:00.0"))
         .go();
   }
 
@@ -179,7 +180,7 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
         .sqlQuery("select to_utc_timestamp('1970-01-01 00:00:00','PST') as UTC_TIMESTAMP from (VALUES(1))")
         .unOrdered()
         .baselineColumns("UTC_TIMESTAMP")
-        .baselineValues(DateTime.parse("1970-01-01T08:00:00.0"))
+        .baselineValues(LocalDateTime.parse("1970-01-01T08:00:00.0"))
         .go();
   }
 
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
index 1b77748..25393e7 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
@@ -17,18 +17,25 @@
  */
 package org.apache.drill.exec.hive;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.UserProtos;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.joda.time.DateTime;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -36,16 +43,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.List;
-import java.util.Map;
-
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 
 @Category({SlowTest.class, HiveStorageTest.class})
 public class TestHiveStorage extends HiveTestBase {
@@ -149,8 +148,8 @@ public class TestHiveStorage extends HiveTestBase {
             3455,
             "stringfield",
             "varcharfield",
-            new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()),
-            new DateTime(Date.valueOf("2013-07-05").getTime()),
+            DateUtility.parseBest("2013-07-05 17:01:00"),
+            DateUtility.parseLocalDate("2013-07-05"),
             "charfield",
             // There is a regression in Hive 1.2.1 in binary type partition columns. Disable for now.
             //"binary",
@@ -168,8 +167,8 @@ public class TestHiveStorage extends HiveTestBase {
             3455,
             "string",
             "varchar",
-            new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()),
-            new DateTime(Date.valueOf("2013-07-05").getTime()),
+            DateUtility.parseBest("2013-07-05 17:01:00"),
+            DateUtility.parseLocalDate("2013-07-05"),
             "char")
         .baselineValues( // All fields are null, but partition fields have non-null values
             null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
@@ -189,8 +188,8 @@ public class TestHiveStorage extends HiveTestBase {
             3455,
             "string",
             "varchar",
-            new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()),
-            new DateTime(Date.valueOf("2013-07-05").getTime()),
+            DateUtility.parseBest("2013-07-05 17:01:00"),
+            DateUtility.parseLocalDate("2013-07-05"),
             "char")
         .go();
   }
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
index f744760..e2e4089 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
@@ -19,10 +19,9 @@ package org.apache.drill.exec.store.jdbc;
 
 import org.apache.drill.categories.JdbcStorageTest;
 import org.apache.drill.PlanTestBase;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.proto.UserBitShared;
 
-import org.joda.time.DateTime;
-
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -58,16 +57,15 @@ public class TestJdbcPluginWithDerbyIT extends PlanTestBase {
                     "FLOAT_FIELD", "REAL_FIELD", "TIME_FIELD", "TIMESTAMP_FIELD", "DATE_FIELD", "CLOB_FIELD")
             .baselineValues(1, "first_name_1", "last_name_1", "1401 John F Kennedy Blvd",   "Philadelphia",     "PA",
                             19107, "{ a : 5, b : 6 }",            123456L,         1, 10.01, false, 1.0, 1.1, 111.00,
-                            new DateTime(1970, 1, 1, 13, 0, 1), new DateTime(2012, 2, 29, 13, 0, 1), new DateTime(2012, 2, 29, 0, 0, 0), "some clob data 1")
+                            DateUtility.parseLocalTime("13:00:01.0"), DateUtility.parseLocalDateTime("2012-02-29 13:00:01.0"), DateUtility.parseLocalDate("2012-02-29"), "some clob data 1")
             .baselineValues(2, "first_name_2", "last_name_2", "One Ferry Building",         "San Francisco",    "CA",
                             94111, "{ foo : \"abc\" }",            95949L,         2, 20.02, true, 2.0, 2.1, 222.00,
-                            new DateTime(1970, 1, 1, 23, 59, 59),  new DateTime(1999, 9, 9, 23, 59, 59), new DateTime(1999, 9, 9, 0, 0, 0), "some more clob data")
+                            DateUtility.parseLocalTime("23:59:59.0"),  DateUtility.parseLocalDateTime("1999-09-09 23:59:59.0"), DateUtility.parseLocalDate("1999-09-09"), "some more clob data")
             .baselineValues(3, "first_name_3", "last_name_3", "176 Bowery",                 "New York",         "NY",
                             10012, "{ z : [ 1, 2, 3 ] }",           45456L,        3, 30.04, true, 3.0, 3.1, 333.00,
-                            new DateTime(1970, 1, 1, 11, 34, 21),  new DateTime(2011, 10, 30, 11, 34, 21), new DateTime(2011, 10, 30, 0, 0, 0), "clobber")
+                            DateUtility.parseLocalTime("11:34:21.0"),  DateUtility.parseLocalDateTime("2011-10-30 11:34:21.0"), DateUtility.parseLocalDate("2011-10-30"), "clobber")
             .baselineValues(4, null, null, "2 15th St NW", "Washington", "DC", 20007, "{ z : { a : 1, b : 2, c : 3 } " +
-                    "}", -67L, 4, 40.04, false, 4.0, 4.1, 444.00, new DateTime(1970, 1, 1, 16, 0, 1), new DateTime
-                    (2015, 6, 1, 16, 0, 1),  new DateTime(2015, 6, 1, 0, 0, 0), "xxx")
+                    "}", -67L, 4, 40.04, false, 4.0, 4.1, 444.00, DateUtility.parseLocalTime("16:00:01.0"), DateUtility.parseLocalDateTime("2015-06-01 16:00:01.0"),  DateUtility.parseLocalDate("2015-06-01"), "xxx")
             .baselineValues(5, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
                             null, null, null, null)
             .build().run();
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
index 7b8c21a..1640913 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
@@ -18,10 +18,9 @@
 package org.apache.drill.exec.store.jdbc;
 
 import org.apache.drill.categories.JdbcStorageTest;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.PlanTestBase;
 
-import org.joda.time.DateTime;
-
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -40,7 +39,7 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
                             "first_name, last_name, address, city, state, zip, " +
                             "bigint_field, smallint_field, numeric_field, " +
                             "boolean_field, double_field, float_field, real_field, " +
-                            "date_field, datetime_field, year_field, " +
+                            "date_field, datetime_field, year_field, time_field, " +
                             "json, text_field, tiny_text_field, medium_text_field, long_text_field, " +
                             "blob_field, bit_field, enum_field " +
                     "from mysql.`drill_mysql_test`.person")
@@ -50,7 +49,7 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
                     "bigint_field", "smallint_field", "numeric_field",
                     "boolean_field",
                     "double_field", "float_field", "real_field",
-                    "date_field", "datetime_field", "year_field",
+                    "date_field", "datetime_field", "year_field", "time_field",
                     "json", "text_field", "tiny_text_field", "medium_text_field", "long_text_field",
                     "blob_field", "bit_field", "enum_field")
             .baselineValues(1,
@@ -58,7 +57,7 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
                     123456789L, 1, 10.01,
                     false,
                     1.0, 1.1, 1.2,
-                    new DateTime(2012, 2, 29, 0, 0, 0), new DateTime(2012, 2, 29, 13, 0, 1), new DateTime(2015, 1, 1, 0, 0, 0),
+                    DateUtility.parseLocalDate("2012-02-29"), DateUtility.parseLocalDateTime("2012-02-29 13:00:01.0"), DateUtility.parseLocalDate("2015-01-01"), DateUtility.parseLocalTime("13:00:01.0"),
                     "{ a : 5, b : 6 }",
                     "It is a long established fact that a reader will be distracted by the readable content of a page when looking at its layout",
                     "xxx",
@@ -71,7 +70,7 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
                     45456767L, 3, 30.04,
                     true,
                     3.0, 3.1, 3.2,
-                    new DateTime(2011, 10, 30, 0, 0, 0), new DateTime(2011, 10, 30, 11, 34, 21), new DateTime(2015, 1, 1, 0, 0, 0),
+                    DateUtility.parseLocalDate("2011-10-30"), DateUtility.parseLocalDateTime("2011-10-30 11:34:21.0"), DateUtility.parseLocalDate("2015-01-01"), DateUtility.parseLocalTime("11:34:21.0"),
                     "{ z : [ 1, 2, 3 ] }",
                     "It is a long established fact that a reader will be distracted by the readable content of a page when looking at its layout",
                     "abc",
@@ -84,7 +83,7 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
                     123090L, -3, 55.12,
                     false,
                     5.0, 5.1, 5.55,
-                    new DateTime(2015, 6, 1, 0, 0, 0), new DateTime(2015, 9, 22, 15, 46, 10), new DateTime(1901, 1, 1, 0, 0, 0),
+                    DateUtility.parseLocalDate("2015-06-01"), DateUtility.parseLocalDateTime("2015-09-22 15:46:10.0"), DateUtility.parseLocalDate("1901-01-01"), DateUtility.parseLocalTime("16:00:01.0"),
                     "{ [ a, b, c ] }",
                     "Neque porro quisquam est qui dolorem ipsum quia dolor sit amet, consectetur, adipisci velit",
                     "abc",
@@ -97,7 +96,7 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
                     null, null, null,
                     null,
                     null, null, null,
-                    null, null, null,
+                    null, null, null, null,
                     null,
                     null,
                     null,
diff --git a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java b/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
index 891666a..5b26be6 100644
--- a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
+++ b/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
@@ -71,11 +71,11 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
   public void eval() {
 
       <#if type.from == "Time">
-      org.joda.time.LocalTime temp = new org.joda.time.LocalTime(in.value, org.joda.time.DateTimeZone.UTC);
-      String str = temp.toString();
+      java.time.LocalDateTime temp = java.time.Instant.ofEpochMilli(in.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+      String str = org.apache.drill.exec.expr.fn.impl.DateUtility.format${type.from}.format(temp);
       <#else>
-      org.joda.time.MutableDateTime temp = new org.joda.time.MutableDateTime(in.value, org.joda.time.DateTimeZone.UTC);
-      String str = org.apache.drill.exec.expr.fn.impl.DateUtility.format${type.from}.print(temp);
+      java.time.LocalDateTime temp = java.time.Instant.ofEpochMilli(in.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+      String str = org.apache.drill.exec.expr.fn.impl.DateUtility.format${type.from}.format(temp);
       </#if>
       out.buffer = buffer;
       out.start = 0;
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java b/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
index d035a99..8d1ab7d 100644
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
+++ b/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
@@ -67,12 +67,12 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
       out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.getDate(in.buffer, in.start, in.end);
 
       <#elseif type.to == "TimeStamp">
-      org.joda.time.format.DateTimeFormatter f = org.apache.drill.exec.expr.fn.impl.DateUtility.getDateTimeFormatter();
-      out.value = org.joda.time.DateTime.parse(input, f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
+      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">
-      org.joda.time.format.DateTimeFormatter f = org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeFormatter();
-      out.value = (int) ((f.parseDateTime(input)).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+      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>
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
index d6d4b1c..07accf1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.expr.fn.impl;
 
-import io.netty.buffer.DrillBuf;
-
 import javax.inject.Inject;
 
 import org.apache.drill.exec.expr.DrillSimpleFunc;
@@ -39,6 +37,8 @@ import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.ops.ContextInformation;
 
+import io.netty.buffer.DrillBuf;
+
 public class DateTypeFunctions {
 
     /**
@@ -280,8 +280,8 @@ public class DateTypeFunctions {
 
         @Override
         public void eval() {
-            org.joda.time.DateTime temp = new org.joda.time.DateTime();
-            String str = org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStampTZ.print(temp);
+            java.time.ZonedDateTime temp = java.time.ZonedDateTime.now();
+            String str = org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStampTZ.format(temp);
             out.buffer = buffer;
             out.start = 0;
             out.end = Math.min(100,  str.length()); // truncate if target type has length smaller than that of input's string
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
index 7b82ff3..f6f7273 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
@@ -17,11 +17,15 @@
  */
 package org.apache.drill.exec.planner.common;
 
-import com.fasterxml.jackson.core.JsonLocation;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.util.TokenBuffer;
-import com.google.common.collect.ImmutableList;
+import static org.apache.drill.exec.planner.logical.DrillOptiq.isLiteralNull;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.core.Values;
@@ -36,11 +40,11 @@ import org.joda.time.DateTime;
 import org.joda.time.DateTimeConstants;
 import org.joda.time.Period;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.List;
-
-import static org.apache.drill.exec.planner.logical.DrillOptiq.isLiteralNull;
+import com.fasterxml.jackson.core.JsonLocation;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.util.TokenBuffer;
+import com.google.common.collect.ImmutableList;
 
 /**
  * Base class for logical and physical Values implemented in Drill.
@@ -197,7 +201,7 @@ public abstract class DrillValuesRelBase extends Values implements DrillRelNode
         if (isLiteralNull(literal)) {
           out.writeDateNull();
         } else {
-          out.writeDate(new DateTime(literal.getValue()));
+          out.writeDate(LocalDateTime.ofInstant(Instant.ofEpochMilli(new DateTime(literal.getValue()).getMillis()), ZoneOffset.UTC).toLocalDate());
         }
         return;
 
@@ -205,7 +209,7 @@ public abstract class DrillValuesRelBase extends Values implements DrillRelNode
         if (isLiteralNull(literal)) {
           out.writeTimeNull();
         } else {
-          out.writeTime(new DateTime(literal.getValue()));
+          out.writeTime(LocalDateTime.ofInstant(Instant.ofEpochMilli(new DateTime(literal.getValue()).getMillis()), ZoneOffset.UTC).toLocalTime());
         }
         return;
 
@@ -213,7 +217,7 @@ public abstract class DrillValuesRelBase extends Values implements DrillRelNode
         if (isLiteralNull(literal)) {
           out.writeTimestampNull();
         } else {
-          out.writeTimestamp(new DateTime(literal.getValue()));
+          out.writeTimestamp(LocalDateTime.ofInstant(Instant.ofEpochMilli(new DateTime(literal.getValue()).getMillis()), ZoneOffset.UTC));
         }
         return;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
index b01413e..786f62b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.store.bson;
 
-import io.netty.buffer.DrillBuf;
-
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
@@ -38,8 +36,7 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
-import org.apache.drill.exec.vector.complex.writer.DateWriter;
-import org.apache.drill.exec.vector.complex.writer.TimeWriter;
+import org.apache.drill.exec.vector.complex.writer.TimeStampWriter;
 import org.bson.BsonBinary;
 import org.bson.BsonReader;
 import org.bson.BsonType;
@@ -47,6 +44,8 @@ import org.joda.time.DateTime;
 
 import com.google.common.base.Preconditions;
 
+import io.netty.buffer.DrillBuf;
+
 public class BsonRecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BsonRecordReader.class);
   public final static int MAX_RECORD_SIZE = 128 * 1024;
@@ -247,14 +246,14 @@ public class BsonRecordReader {
   }
 
   private void writeTimeStamp(int timestamp, final MapOrListWriterImpl writer, String fieldName, boolean isList) {
-    DateTime dateTime = new DateTime(timestamp);
-    TimeWriter t;
+    DateTime dateTime = new DateTime(timestamp*1000L);
+    TimeStampWriter t;
     if (isList == false) {
-      t = writer.map.time(fieldName);
+      t = writer.map.timeStamp(fieldName);
     } else {
-      t = writer.list.time();
+      t = writer.list.timeStamp();
     }
-    t.writeTime((int) (dateTime.withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis()));
+    t.writeTimeStamp((int) (dateTime.withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis()));
   }
 
   private void writeString(String readString, final MapOrListWriterImpl writer, String fieldName, boolean isList) {
@@ -297,13 +296,13 @@ public class BsonRecordReader {
 
   private void writeDateTime(long readDateTime, final MapOrListWriterImpl writer, String fieldName, boolean isList) {
     DateTime date = new DateTime(readDateTime);
-    DateWriter dt;
+    TimeStampWriter dt;
     if (isList == false) {
-      dt = writer.map.date(fieldName);
+      dt = writer.map.timeStamp(fieldName);
     } else {
-      dt = writer.list.date();
+      dt = writer.list.timeStamp();
     }
-    dt.writeDate(date.withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+    dt.writeTimeStamp(date.withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
   }
 
   private void writeBoolean(boolean readBoolean, final MapOrListWriterImpl writer, String fieldName, boolean isList) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
index 018653b..ac6f001 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
@@ -17,22 +17,24 @@
  */
 package org.apache.drill.exec.util;
 
+import java.time.LocalDateTime;
 import java.util.List;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.util.DrillStringUtils;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
 public class VectorUtil {
 
   public static final int DEFAULT_COLUMN_WIDTH = 15;
@@ -114,6 +116,8 @@ public class VectorUtil {
           // TODO(DRILL-3882) - remove this once the datetime is not returned in an
           // object needlessly holding a timezone
           rowValues.add(DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSS").print((DateTime) o));
+        } else if (o instanceof LocalDateTime) {
+          rowValues.add(DateUtility.formatTimeStamp.format((LocalDateTime) o));
         } else {
           rowValues.add(o.toString());
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java
index 8ac8df8..9d7c569 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java
@@ -19,14 +19,12 @@ package org.apache.drill.exec.vector.complex.fn;
 
 import java.io.IOException;
 import java.math.BigDecimal;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.TemporalAccessor;
 
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
-import org.joda.time.DateTime;
-import org.joda.time.DateTimeZone;
 import org.joda.time.Period;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.ISODateTimeFormat;
 import org.joda.time.format.ISOPeriodFormat;
 
 import com.fasterxml.jackson.core.JsonGenerator;
@@ -61,9 +59,9 @@ public class BasicJsonOutput implements JsonOutput {
       break;
     }
     case ISO: {
-      dateFormatter = ISODateTimeFormat.date();
-      timeFormatter = ISODateTimeFormat.time();
-      timestampFormatter = ISODateTimeFormat.dateTime();
+      dateFormatter = DateUtility.isoFormatDate;
+      timeFormatter = DateUtility.isoFormatTime;
+      timestampFormatter = DateUtility.isoFormatTimeStamp;
       break;
     }
 
@@ -209,7 +207,7 @@ public class BasicJsonOutput implements JsonOutput {
   @Override
   public void writeDate(FieldReader reader) throws IOException {
     if (reader.isSet()) {
-      writeDate(reader.readDateTime());
+      writeDate(reader.readLocalDate());
     } else {
       writeDateNull();
     }
@@ -218,7 +216,7 @@ public class BasicJsonOutput implements JsonOutput {
   @Override
   public void writeTime(FieldReader reader) throws IOException {
     if (reader.isSet()) {
-      writeTime(reader.readDateTime());
+      writeTime(reader.readLocalTime());
     } else {
       writeTimeNull();
     }
@@ -227,7 +225,7 @@ public class BasicJsonOutput implements JsonOutput {
   @Override
   public void writeTimestamp(FieldReader reader) throws IOException {
     if (reader.isSet()) {
-      writeTimestamp(reader.readDateTime());
+      writeTimestamp(reader.readLocalDateTime());
     } else {
       writeTimeNull();
     }
@@ -335,7 +333,7 @@ public class BasicJsonOutput implements JsonOutput {
   @Override
   public void writeDate(int index, FieldReader reader) throws IOException {
     if (reader.isSet()) {
-      writeDate(reader.readDateTime(index));
+      writeDate(reader.readLocalDate(index));
     } else {
       writeDateNull();
     }
@@ -344,7 +342,7 @@ public class BasicJsonOutput implements JsonOutput {
   @Override
   public void writeTime(int index, FieldReader reader) throws IOException {
     if (reader.isSet()) {
-      writeTime(reader.readDateTime(index));
+      writeTime(reader.readLocalTime(index));
     } else {
       writeTimeNull();
     }
@@ -353,7 +351,7 @@ public class BasicJsonOutput implements JsonOutput {
   @Override
   public void writeTimestamp(int index, FieldReader reader) throws IOException {
     if (reader.isSet()) {
-      writeTimestamp(reader.readDateTime(index));
+      writeTimestamp(reader.readLocalDateTime(index));
     } else {
       writeTimestampNull();
     }
@@ -433,18 +431,18 @@ public class BasicJsonOutput implements JsonOutput {
   }
 
   @Override
-  public void writeDate(DateTime value) throws IOException {
-    gen.writeString(dateFormatter.print(value.withZone(DateTimeZone.UTC)));
+  public void writeDate(TemporalAccessor value) throws IOException {
+    gen.writeString(dateFormatter.format(value));
   }
 
   @Override
-  public void writeTime(DateTime value) throws IOException {
-    gen.writeString(timeFormatter.print(value.withZone(DateTimeZone.UTC)));
+  public void writeTime(TemporalAccessor value) throws IOException {
+    gen.writeString(timeFormatter.format(value));
   }
 
   @Override
-  public void writeTimestamp(DateTime value) throws IOException {
-    gen.writeString(timestampFormatter.print(value.withZone(DateTimeZone.UTC)));
+  public void writeTimestamp(TemporalAccessor value) throws IOException {
+    gen.writeString(timestampFormatter.format(value));
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
index b1eea06..9b509e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
@@ -19,8 +19,11 @@ package org.apache.drill.exec.vector.complex.fn;
 
 import java.io.IOException;
 import java.math.BigDecimal;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
+import java.time.temporal.TemporalAccessor;
 
-import org.joda.time.DateTime;
 import org.joda.time.Period;
 
 import com.fasterxml.jackson.core.JsonGenerator;
@@ -52,7 +55,7 @@ public class ExtendedJsonOutput extends BasicJsonOutput {
   }
 
   @Override
-  public void writeDate(DateTime value) throws IOException {
+  public void writeDate(TemporalAccessor value) throws IOException {
     gen.writeStartObject();
     gen.writeFieldName(ExtendedType.DATE.serialized);
     super.writeDate(value);
@@ -60,18 +63,18 @@ public class ExtendedJsonOutput extends BasicJsonOutput {
   }
 
   @Override
-  public void writeTime(DateTime value) throws IOException {
+  public void writeTime(TemporalAccessor value) throws IOException {
     gen.writeStartObject();
     gen.writeFieldName(ExtendedType.TIME.serialized);
-    super.writeTime(value);
+    super.writeTime(((LocalTime) value).atOffset(ZoneOffset.UTC));   // output time in local time zone
     gen.writeEndObject();
   }
 
   @Override
-  public void writeTimestamp(DateTime value) throws IOException {
+  public void writeTimestamp(TemporalAccessor value) throws IOException {
     gen.writeStartObject();
     gen.writeFieldName(ExtendedType.TIMESTAMP.serialized);
-    super.writeTimestamp(value);
+    super.writeTimestamp(((LocalDateTime) value).atOffset(ZoneOffset.UTC)); // output date time in local time zone
     gen.writeEndObject();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java
index a921142..572386b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java
@@ -19,9 +19,9 @@ package org.apache.drill.exec.vector.complex.fn;
 
 import java.io.IOException;
 import java.math.BigDecimal;
+import java.time.temporal.TemporalAccessor;
 
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
-import org.joda.time.DateTime;
 import org.joda.time.Period;
 
 /**
@@ -52,9 +52,9 @@ public interface JsonOutput {
   void writeVar16Char(String value) throws IOException;
   void writeBinary(byte[] value) throws IOException;
   void writeBoolean(boolean value) throws IOException;
-  void writeDate(DateTime value) throws IOException;
-  void writeTime(DateTime value) throws IOException;
-  void writeTimestamp(DateTime value) throws IOException;
+  void writeDate(TemporalAccessor value) throws IOException;
+  void writeTime(TemporalAccessor value) throws IOException;
+  void writeTimestamp(TemporalAccessor value) throws IOException;
   void writeInterval(Period value) throws IOException;
   void writeDecimalNull() throws IOException;
   void writeTinyIntNull() throws IOException;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
index fec9e66..9f0ac8a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
@@ -18,8 +18,14 @@
 package org.apache.drill.exec.vector.complex.fn;
 
 import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.OffsetTime;
+import java.time.ZoneOffset;
 
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
@@ -40,8 +46,6 @@ import org.apache.drill.exec.vector.complex.writer.TimeWriter;
 import org.apache.drill.exec.vector.complex.writer.VarBinaryWriter;
 import org.joda.time.DateTime;
 import org.joda.time.Period;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.ISODateTimeFormat;
 import org.joda.time.format.ISOPeriodFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -227,8 +231,9 @@ abstract class VectorOutput {
     public void writeTime(boolean isNull) throws IOException {
       TimeWriter t = writer.time();
       if(!isNull){
-        DateTimeFormatter f = ISODateTimeFormat.time();
-        t.writeTime((int) ((f.parseDateTime(parser.getValueAsString())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis()));
+        // read time and obtain the local time in the provided time zone.
+        LocalTime localTime = OffsetTime.parse(parser.getValueAsString(), DateUtility.isoFormatTime).toLocalTime();
+        t.writeTime((int) ((localTime.toNanoOfDay() + 500000L) / 1000000L)); // round to milliseconds
       }
     }
 
@@ -242,8 +247,9 @@ abstract class VectorOutput {
           ts.writeTimeStamp(dt.getMillis());
           break;
         case VALUE_STRING:
-          DateTimeFormatter f = ISODateTimeFormat.dateTime();
-          ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+          OffsetDateTime originalDateTime = OffsetDateTime.parse(parser.getValueAsString(), DateUtility.isoFormatTimeStamp);
+          OffsetDateTime utcDateTime = OffsetDateTime.of(originalDateTime.toLocalDateTime(), ZoneOffset.UTC);   // strips the time zone from the original
+          ts.writeTimeStamp(utcDateTime.toInstant().toEpochMilli());
           break;
         default:
           throw UserException.unsupportedError()
@@ -319,9 +325,10 @@ abstract class VectorOutput {
     public void writeDate(boolean isNull) throws IOException {
       DateWriter dt = writer.date(fieldName);
       if(!isNull){
-        DateTimeFormatter f = ISODateTimeFormat.date();
-        DateTime date = f.parseDateTime(parser.getValueAsString());
-        dt.writeDate(date.withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+        LocalDate    localDate = LocalDate.parse(parser.getValueAsString(), DateUtility.isoFormatDate);
+        OffsetDateTime utcDate = OffsetDateTime.of(localDate, LocalTime.MIDNIGHT, ZoneOffset.UTC);
+
+        dt.writeDate(utcDate.toInstant().toEpochMilli()); // round to milliseconds
       }
     }
 
@@ -330,8 +337,8 @@ abstract class VectorOutput {
       @SuppressWarnings("resource")
       TimeWriter t = writer.time(fieldName);
       if(!isNull){
-        DateTimeFormatter f = ISODateTimeFormat.time();
-        t.writeTime((int) ((f.parseDateTime(parser.getValueAsString())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis()));
+        LocalTime localTime = OffsetTime.parse(parser.getValueAsString(), DateUtility.isoFormatTime).toLocalTime();
+        t.writeTime((int) ((localTime.toNanoOfDay() + 500000L) / 1000000L)); // round to milliseconds
       }
     }
 
@@ -346,8 +353,9 @@ abstract class VectorOutput {
           ts.writeTimeStamp(dt.getMillis());
           break;
         case VALUE_STRING:
-          DateTimeFormatter f = ISODateTimeFormat.dateTime();
-          ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+          OffsetDateTime originalDateTime = OffsetDateTime.parse(parser.getValueAsString(), DateUtility.isoFormatTimeStamp);
+          OffsetDateTime utcDateTime = OffsetDateTime.of(originalDateTime.toLocalDateTime(), ZoneOffset.UTC);   // strips the time zone from the original
+          ts.writeTimeStamp(utcDateTime.toInstant().toEpochMilli());
           break;
         default:
           throw UserException.unsupportedError()
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
index 9986579..8047949 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
@@ -17,21 +17,23 @@
  */
 package org.apache.drill;
 
+import static org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStamp;
+
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
-import org.joda.time.DateTime;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.math.BigDecimal;
-
-import static org.apache.drill.exec.expr.fn.impl.DateUtility.formatDate;
-import static org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStamp;
-
 @Category(SqlFunctionTest.class)
 public class TestFunctionsQuery extends BaseTestQuery {
 
@@ -552,7 +554,7 @@ public class TestFunctionsQuery extends BaseTestQuery {
         "timestamp '2008-2-23 12:23:23' as TS " +
         "FROM cp.`tpch/region.parquet` limit 1";
 
-    DateTime date = formatTimeStamp.parseDateTime("2008-02-23 12:23:23.0");
+    LocalDateTime date = LocalDateTime.parse("2008-02-23 12:23:23.0", formatTimeStamp);
     testBuilder()
         .sqlQuery(query)
         .unOrdered()
@@ -694,7 +696,7 @@ public class TestFunctionsQuery extends BaseTestQuery {
         "To_DaTe('2003/07/09', 'yyyy/MM/dd') as col3 " +
         "from cp.`employee.json` LIMIT 1";
 
-    DateTime date = formatDate.parseDateTime("2003-07-09");
+    LocalDate date = LocalDate.parse("2003-07-09");
 
     testBuilder()
         .sqlQuery(query)
@@ -761,8 +763,8 @@ public class TestFunctionsQuery extends BaseTestQuery {
     String query = "select to_timestamp(cast('800120400.12312' as decimal(38, 5))) as DEC38_TS, to_timestamp(200120400) as INT_TS\n" +
         "from cp.`employee.json` where employee_id < 2";
 
-    DateTime result1 = new DateTime(800120400123L);
-    DateTime result2 = new DateTime(200120400000L);
+    LocalDateTime result1 = Instant.ofEpochMilli(800120400123L).atZone(ZoneOffset.systemDefault()).toLocalDateTime();
+    LocalDateTime result2 = Instant.ofEpochMilli(200120400000L).atZone(ZoneOffset.systemDefault()).toLocalDateTime();
 
     testBuilder()
         .sqlQuery(query)
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
index b9e1752..d33fbee 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
@@ -427,7 +427,7 @@ public class TestStarQueries extends BaseTestQuery {
   @Test // DRILL-1500
   @Category(UnlikelyTest.class)
   public void testStarPartitionFilterOrderBy() throws Exception {
-    org.joda.time.DateTime mydate = new org.joda.time.DateTime("1994-01-20T00:00:00.000");
+    java.time.LocalDate mydate = java.time.LocalDate.parse("1994-01-20");
 
     testBuilder()
     .sqlQuery("select * from dfs.`multilevel/parquet` where dir0=1994 and dir1='Q1' order by dir0 limit 1")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
index 4eb3917..01ceb90 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
@@ -17,23 +17,26 @@
  */
 package org.apache.drill.exec.fn.impl;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import mockit.integration.junit4.JMockit;
-import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.apache.drill.test.BaseTestQuery;
-import org.apache.drill.categories.SqlFunctionTest;
-import org.apache.drill.categories.UnlikelyTest;
-import org.joda.time.DateTime;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
 import java.math.BigDecimal;
 import java.math.MathContext;
 import java.math.RoundingMode;
+import java.time.LocalDate;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.drill.categories.SqlFunctionTest;
+import org.apache.drill.categories.UnlikelyTest;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.test.BaseTestQuery;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import mockit.integration.junit4.JMockit;
+
 @RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestCastFunctions extends BaseTestQuery {
@@ -84,7 +87,7 @@ public class TestCastFunctions extends BaseTestQuery {
       .sqlQuery(query)
       .ordered()
       .baselineColumns("col")
-      .baselineValues(new DateTime(1969, 12, 31, 0, 0))
+      .baselineValues(LocalDate.of(1969, 12, 31))
       .build()
       .run();
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateAddFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateAddFunctions.java
index b96a32a..20f1995 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateAddFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateAddFunctions.java
@@ -17,10 +17,13 @@
  */
 package org.apache.drill.exec.fn.impl;
 
-import org.apache.drill.test.BaseTestQuery;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
-import org.joda.time.DateTime;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.test.BaseTestQuery;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -41,12 +44,12 @@ public class TestDateAddFunctions extends BaseTestQuery {
       .sqlQuery(query)
       .unOrdered()
       .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6")
-      .baselineValues(DateTime.parse("2015-01-27T07:27:05.0"),
-                      DateTime.parse("2015-01-29T07:27:05.0"),
-                      DateTime.parse("2015-01-24T12:27:05.0"),
-                      DateTime.parse("2015-01-24T07:32:05.0"),
-                      DateTime.parse("2015-01-24T07:27:10.0"),
-                      DateTime.parse("2015-01-29T17:47:35.0"))
+      .baselineValues(LocalDateTime.parse("2015-01-27T07:27:05.0"),
+                      LocalDateTime.parse("2015-01-29T07:27:05.0"),
+                      LocalDateTime.parse("2015-01-24T12:27:05.0"),
+                      LocalDateTime.parse("2015-01-24T07:32:05.0"),
+                      LocalDateTime.parse("2015-01-24T07:27:10.0"),
+                      LocalDateTime.parse("2015-01-29T17:47:35.0"))
       .go();
   }
 
@@ -61,9 +64,9 @@ public class TestDateAddFunctions extends BaseTestQuery {
       .sqlQuery(query)
       .unOrdered()
       .baselineColumns("col1", "col2", "col3")
-      .baselineValues(DateTime.parse("2015-04-24"),
-                      DateTime.parse("2015-06-24"),
-                      DateTime.parse("2020-01-24"))
+      .baselineValues(DateUtility.parseBest("2015-04-24"),
+                      DateUtility.parseBest("2015-06-24"),
+                      DateUtility.parseBest("2020-01-24"))
       .go();
   }
 
@@ -77,8 +80,8 @@ public class TestDateAddFunctions extends BaseTestQuery {
       .sqlQuery(query)
       .unOrdered()
       .baselineColumns("col1", "col2")
-      .baselineValues(DateTime.parse("2015-01-27"),
-                      DateTime.parse("2015-01-29"))
+      .baselineValues(LocalDate.parse("2015-01-27"),
+                      LocalDate.parse("2015-01-29"))
       .go();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateTruncFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateTruncFunctions.java
index 0ef9928..ff32733 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateTruncFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateTruncFunctions.java
@@ -17,18 +17,19 @@
  */
 package org.apache.drill.exec.fn.impl;
 
-import org.apache.drill.test.BaseTestQuery;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
+
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
-import org.joda.time.DateTime;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.test.BaseTestQuery;
 import org.joda.time.Period;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.apache.drill.exec.expr.fn.impl.DateUtility.formatDate;
-import static org.apache.drill.exec.expr.fn.impl.DateUtility.formatTime;
-import static org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStamp;
-
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestDateTruncFunctions extends BaseTestQuery {
 
@@ -52,16 +53,16 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("second", "minute", "hour", "day", "month", "year", "quarter", "decade", "century", "millennium")
         .baselineValues(
-            formatTime.parseDateTime("2:30:21.0"), // seconds
-            formatTime.parseDateTime("2:30:00.0"), // minute
-            formatTime.parseDateTime("2:00:00.0"), // hour
-            formatTime.parseDateTime("0:00:00.0"), // day
-            formatTime.parseDateTime("0:00:00.0"), // month
-            formatTime.parseDateTime("0:00:00.0"), // year
-            formatTime.parseDateTime("0:00:00.0"), // quarter
-            formatTime.parseDateTime("0:00:00.0"), // decade
-            formatTime.parseDateTime("0:00:00.0"), // century
-            formatTime.parseDateTime("0:00:00.0")) // millennium
+            DateUtility.parseLocalTime("2:30:21.0"), // seconds
+            DateUtility.parseLocalTime("2:30:00.0"), // minute
+            DateUtility.parseLocalTime("2:00:00.0"), // hour
+            DateUtility.parseLocalTime("0:00:00.0"), // day
+            DateUtility.parseLocalTime("0:00:00.0"), // month
+            DateUtility.parseLocalTime("0:00:00.0"), // year
+            DateUtility.parseLocalTime("0:00:00.0"), // quarter
+            DateUtility.parseLocalTime("0:00:00.0"), // decade
+            DateUtility.parseLocalTime("0:00:00.0"), // century
+            DateUtility.parseLocalTime("0:00:00.0")) // millennium
         .go();
   }
 
@@ -88,19 +89,19 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("second", "minute", "hour", "day", "month", "week" , "year", "q1", "q2", "q3", "decade1", "decade2", "decade3")
         .baselineValues(
-            formatDate.parseDateTime("2011-02-03"), // seconds
-            formatDate.parseDateTime("2011-02-03"), // minute
-            formatDate.parseDateTime("2011-02-03"), // hour
-            formatDate.parseDateTime("2011-02-03"), // day
-            formatDate.parseDateTime("2011-02-01"), // month
-            formatDate.parseDateTime("2011-01-31"), // week
-            formatDate.parseDateTime("2011-01-01"), // year
-            formatDate.parseDateTime("2011-04-01"), // quarter-1
-            formatDate.parseDateTime("2011-07-01"), // quarter-2
-            formatDate.parseDateTime("2011-07-01"), // quarter-3
-            formatDate.parseDateTime("2010-01-01"), // decade-1
-            formatDate.parseDateTime("2070-01-01"), // decade-2
-            formatDate.parseDateTime("1970-01-01")) // decade-3
+            DateUtility.parseLocalDate("2011-02-03"), // seconds
+            DateUtility.parseLocalDate("2011-02-03"), // minute
+            DateUtility.parseLocalDate("2011-02-03"), // hour
+            DateUtility.parseLocalDate("2011-02-03"), // day
+            DateUtility.parseLocalDate("2011-02-01"), // month
+            DateUtility.parseLocalDate("2011-01-31"), // week
+            DateUtility.parseLocalDate("2011-01-01"), // year
+            DateUtility.parseLocalDate("2011-04-01"), // quarter-1
+            DateUtility.parseLocalDate("2011-07-01"), // quarter-2
+            DateUtility.parseLocalDate("2011-07-01"), // quarter-3
+            DateUtility.parseLocalDate("2010-01-01"), // decade-1
+            DateUtility.parseLocalDate("2070-01-01"), // decade-2
+            DateUtility.parseLocalDate("1970-01-01")) // decade-3
         .go();
   }
 
@@ -121,18 +122,18 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("c1", "c2", "c3", "c4", "c5")
         .baselineValues(
-            formatDate.parseDateTime("2001-01-01"), // c1
-            formatDate.parseDateTime("1901-01-01"), // c2
-            formatDate.parseDateTime("1901-01-01"), // c3
-            formatDate.parseDateTime("0801-01-01"), // c4
-            formatDate.parseDateTime("0001-01-01")) // c5
+            DateUtility.parseLocalDate("2001-01-01"), // c1
+            DateUtility.parseLocalDate("1901-01-01"), // c2
+            DateUtility.parseLocalDate("1901-01-01"), // c3
+            DateUtility.parseLocalDate("0801-01-01"), // c4
+            DateUtility.parseLocalDate("0001-01-01")) // c5
         .go();
   }
 
   @Test
   public void test() throws Exception {
     org.joda.time.MutableDateTime dateTime = new org.joda.time.MutableDateTime(org.joda.time.DateTimeZone.UTC);
-    dateTime.setMillis(formatDate.parseDateTime("2001-01-01"));
+    dateTime.setMillis(DateUtility.parseLocalDate("2001-01-01").atTime(LocalTime.MIDNIGHT).atOffset(ZoneOffset.UTC).toInstant().toEpochMilli());
     dateTime.setRounding(dateTime.getChronology().centuryOfEra());
   }
 
@@ -153,11 +154,11 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("m1", "m2", "m3", "m4", "m5")
         .baselineValues(
-            formatDate.parseDateTime("2001-01-01"), // m1
-            formatDate.parseDateTime("1001-01-01"), // m2
-            formatDate.parseDateTime("1001-01-01"), // m3
-            formatDate.parseDateTime("0001-01-01"), // m4
-            formatDate.parseDateTime("0001-01-01")) // m5
+            DateUtility.parseLocalDate("2001-01-01"), // m1
+            DateUtility.parseLocalDate("1001-01-01"), // m2
+            DateUtility.parseLocalDate("1001-01-01"), // m3
+            DateUtility.parseLocalDate("0001-01-01"), // m4
+            DateUtility.parseLocalDate("0001-01-01")) // m5
         .go();
   }
 
@@ -184,19 +185,19 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("second", "minute", "hour", "day", "month", "week" , "year", "q1", "q2", "q3", "decade1", "decade2", "decade3")
         .baselineValues(
-            formatTimeStamp.parseDateTime("2011-02-03 10:11:12.0"), // seconds
-            formatTimeStamp.parseDateTime("2011-02-03 10:11:00.0"), // minute
-            formatTimeStamp.parseDateTime("2011-02-03 10:00:00.0"), // hour
-            formatTimeStamp.parseDateTime("2011-02-03 00:00:00.0"), // day
-            formatTimeStamp.parseDateTime("2011-02-01 00:00:00.0"), // month
-            formatTimeStamp.parseDateTime("2011-01-31 00:00:00.0"), // week
-            formatTimeStamp.parseDateTime("2011-01-01 00:00:00.0"), // year
-            formatTimeStamp.parseDateTime("2011-04-01 00:00:00.0"), // quarter-1
-            formatTimeStamp.parseDateTime("2011-07-01 00:00:00.0"), // quarter-2
-            formatTimeStamp.parseDateTime("2011-07-01 00:00:00.0"), // quarter-3
-            formatTimeStamp.parseDateTime("2010-01-01 00:00:00.0"), // decade-1
-            formatTimeStamp.parseDateTime("2070-01-01 00:00:00.0"), // decade-2
-            formatTimeStamp.parseDateTime("1970-01-01 00:00:00.0")) // decade-3
+            DateUtility.parseLocalDateTime("2011-02-03 10:11:12.0"), // seconds
+            DateUtility.parseLocalDateTime("2011-02-03 10:11:00.0"), // minute
+            DateUtility.parseLocalDateTime("2011-02-03 10:00:00.0"), // hour
+            DateUtility.parseLocalDateTime("2011-02-03 00:00:00.0"), // day
+            DateUtility.parseLocalDateTime("2011-02-01 00:00:00.0"), // month
+            DateUtility.parseLocalDateTime("2011-01-31 00:00:00.0"), // week
+            DateUtility.parseLocalDateTime("2011-01-01 00:00:00.0"), // year
+            DateUtility.parseLocalDateTime("2011-04-01 00:00:00.0"), // quarter-1
+            DateUtility.parseLocalDateTime("2011-07-01 00:00:00.0"), // quarter-2
+            DateUtility.parseLocalDateTime("2011-07-01 00:00:00.0"), // quarter-3
+            DateUtility.parseLocalDateTime("2010-01-01 00:00:00.0"), // decade-1
+            DateUtility.parseLocalDateTime("2070-01-01 00:00:00.0"), // decade-2
+            DateUtility.parseLocalDateTime("1970-01-01 00:00:00.0")) // decade-3
         .go();
   }
 
@@ -217,11 +218,11 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("c1", "c2", "c3", "c4", "c5")
         .baselineValues(
-            formatTimeStamp.parseDateTime("2001-01-01 00:00:00.0"), // c1
-            formatTimeStamp.parseDateTime("1901-01-01 00:00:00.0"), // c2
-            formatTimeStamp.parseDateTime("1901-01-01 00:00:00.0"), // c3
-            formatTimeStamp.parseDateTime("0801-01-01 00:00:00.0"), // c4
-            formatTimeStamp.parseDateTime("0001-01-01 00:00:00.0")) // c5
+            DateUtility.parseLocalDateTime("2001-01-01 00:00:00.0"), // c1
+            DateUtility.parseLocalDateTime("1901-01-01 00:00:00.0"), // c2
+            DateUtility.parseLocalDateTime("1901-01-01 00:00:00.0"), // c3
+            DateUtility.parseLocalDateTime("0801-01-01 00:00:00.0"), // c4
+            DateUtility.parseLocalDateTime("0001-01-01 00:00:00.0")) // c5
         .go();
   }
 
@@ -242,11 +243,11 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         .unOrdered()
         .baselineColumns("m1", "m2", "m3", "m4", "m5")
         .baselineValues(
-            formatTimeStamp.parseDateTime("2001-01-01 00:00:00.0"), // m1
-            formatTimeStamp.parseDateTime("1001-01-01 00:00:00.0"), // m2
-            formatTimeStamp.parseDateTime("1001-01-01 00:00:00.0"), // m3
-            formatTimeStamp.parseDateTime("0001-01-01 00:00:00.0"), // m4
-            formatTimeStamp.parseDateTime("0001-01-01 00:00:00.0")) // m5
+            DateUtility.parseLocalDateTime("2001-01-01 00:00:00.0"), // m1
+            DateUtility.parseLocalDateTime("1001-01-01 00:00:00.0"), // m2
+            DateUtility.parseLocalDateTime("1001-01-01 00:00:00.0"), // m3
+            DateUtility.parseLocalDateTime("0001-01-01 00:00:00.0"), // m4
+            DateUtility.parseLocalDateTime("0001-01-01 00:00:00.0")) // m5
         .go();
   }
 
@@ -327,12 +328,12 @@ public class TestDateTruncFunctions extends BaseTestQuery {
         + "date_trunc('YEAR', date '2011-2-2') as DATE2 "
         + "from cp.`employee.json` where employee_id < 2";
 
-    DateTime time1 = formatTime.parseDateTime("2:30:00.0");
-    DateTime time2 = formatTime.parseDateTime("2:30:21.0");
-    DateTime ts1 = formatTimeStamp.parseDateTime("1991-05-05 10:00:00.0");
-    DateTime ts2 = formatTimeStamp.parseDateTime("1991-05-05 10:11:12.0");
-    DateTime date1 = formatDate.parseDateTime("2011-02-01");
-    DateTime date2 = formatDate.parseDateTime("2011-01-01");
+    LocalTime time1 = DateUtility.parseLocalTime("2:30:00.0");
+    LocalTime time2 = DateUtility.parseLocalTime("2:30:21.0");
+    LocalDateTime ts1 = DateUtility.parseLocalDateTime("1991-05-05 10:00:00.0");
+    LocalDateTime ts2 = DateUtility.parseLocalDateTime("1991-05-05 10:11:12.0");
+    LocalDate date1 = DateUtility.parseLocalDate("2011-02-01");
+    LocalDate date2 = DateUtility.parseLocalDate("2011-01-01");
 
     testBuilder()
         .sqlQuery(query)
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java
index a546a50..5cb28e5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java
@@ -17,9 +17,11 @@
  */
 package org.apache.drill.exec.fn.impl;
 
-import org.apache.drill.test.BaseTestQuery;
+import java.time.LocalDate;
+
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
+import org.apache.drill.test.BaseTestQuery;
 import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
@@ -53,9 +55,9 @@ public class TestNewDateFunctions extends BaseTestQuery {
         .sqlQuery("select case when isdate(date1) then cast(date1 as date) else null end res1 from " + dateValues)
         .unOrdered()
         .baselineColumns("res1")
-        .baselineValues(new DateTime(1900, 1, 1, 0, 0))
-        .baselineValues(new DateTime(3500, 1, 1, 0, 0))
-        .baselineValues(new DateTime(2000, 12, 31, 0, 0))
+        .baselineValues(LocalDate.of(1900, 1, 1))
+        .baselineValues(LocalDate.of(3500, 1, 1))
+        .baselineValues(LocalDate.of(2000, 12, 31))
         .baselineValues(new Object[] {null})
         .baselineValues(new Object[] {null})
         .baselineValues(new Object[] {null})
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index 493a3b6..1c80579 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -20,6 +20,9 @@ package org.apache.drill.exec.fn.interp;
 import static org.junit.Assert.assertEquals;
 
 import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
 import java.util.List;
 
 import org.apache.drill.categories.SlowTest;
@@ -48,12 +51,11 @@ import org.apache.drill.exec.store.mock.MockScanBatchCreator;
 import org.apache.drill.exec.store.mock.MockSubScanPOP;
 import org.apache.drill.exec.store.mock.MockTableDef;
 import org.apache.drill.exec.vector.ValueVector;
-import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
-import org.junit.experimental.categories.Category;
 
 @Category({SlowTest.class, SqlTest.class})
 public class ExpressionInterpreterTest  extends PopUnitTestBase {
@@ -141,7 +143,7 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     final ByteBuffer buffer = ByteBuffer.allocate(12);
     buffer.putLong(out.value);
     final long l = buffer.getLong(0);
-    final DateTime t = new DateTime(l);
+    final LocalDateTime t = Instant.ofEpochMilli(l).atZone(ZoneOffset.systemDefault()).toLocalDateTime();
 
     final String[] expectedFirstTwoValues = {t.toString(), t.toString()};
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index 06d761f..581c972 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -25,11 +25,11 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.time.LocalDate;
+import java.time.LocalTime;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.drill.test.BaseTestQuery;
-import org.apache.drill.test.QueryTestUtil;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.ClassTransformer.ScalarReplacementOption;
@@ -44,16 +44,17 @@ import org.apache.drill.exec.util.ByteBufUtil.HadoopWritables;
 import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarCharVector;
-import org.joda.time.DateTime;
+import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.test.QueryTestUtil;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
 import io.netty.buffer.DrillBuf;
-import org.junit.experimental.categories.Category;
 
 @Category(UnlikelyTest.class)
 public class TestConvertFunctions extends BaseTestQuery {
@@ -66,8 +67,8 @@ public class TestConvertFunctions extends BaseTestQuery {
   private static final String DATE_TIME_BE = "\\x00\\x00\\x00\\x49\\x77\\x85\\x1f\\x8e";
   private static final String DATE_TIME_LE = "\\x8e\\x1f\\x85\\x77\\x49\\x00\\x00\\x00";
 
-  private static DateTime time = DateTime.parse("01:23:45.678", DateUtility.getTimeFormatter());
-  private static DateTime date = DateTime.parse("1980-01-01", DateUtility.getDateTimeFormatter());
+  private static LocalTime time = LocalTime.parse("01:23:45.678", DateUtility.getTimeFormatter());
+  private static LocalDate date = LocalDate.parse("1980-01-01", DateUtility.getDateTimeFormatter());
 
   String textFileContent;
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestNestedDateTimeTimestamp.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestNestedDateTimeTimestamp.java
new file mode 100644
index 0000000..6a0aa78
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestNestedDateTimeTimestamp.java
@@ -0,0 +1,234 @@
+/*
+ * 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.exec.physical.impl;
+
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.test.TestBuilder;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * For DRILL-6242, output for Date, Time, Timestamp should use different classes
+ */
+public class TestNestedDateTimeTimestamp extends BaseTestQuery {
+  private static final String              DATAFILE       = "cp.`datetime.parquet`";
+  private static final Map<String, Object> expectedRecord = new TreeMap<String, Object>();
+
+  static {
+    /**
+     * Data in the parquet file represents this equivalent JSON, but with typed
+     * data, time, and timestamps: { "date" : "1970-01-11", "time" :
+     * "00:00:03.600", "timestamp" : "2018-03-23T17:40:52.123Z", "date_list" : [
+     * "1970-01-11" ], "time_list" : [ "00:00:03.600" ], "timestamp_list" : [
+     * "2018-03-23T17:40:52.123Z" ], "time_map" : { "date" : "1970-01-11",
+     * "time" : "00:00:03.600", "timestamp" : "2018-03-23T17:40:52.123Z" } }
+     *
+     * Note that when the above data is read in to Drill, Drill modifies the
+     * timestamp to local time zone, and preserving the <date> and <time>
+     * values. This effectively changes the timestamp, if the time zone is not
+     * UTC.
+     */
+
+    LocalDate date = DateUtility.parseLocalDate("1970-01-11");
+    LocalTime time = DateUtility.parseLocalTime("00:00:03.600");
+    LocalDateTime timestamp = DateUtility.parseLocalDateTime("2018-03-23 17:40:52.123");
+    expectedRecord.put("`date`", date);
+    expectedRecord.put("`time`", time);
+    expectedRecord.put("`timestamp`", timestamp);
+    expectedRecord.put("`date_list`", Arrays.asList(date));
+    expectedRecord.put("`time_list`", Arrays.asList(time));
+    expectedRecord.put("`timestamp_list`", Arrays.asList(timestamp));
+    Map<String, Object> nestedMap = new TreeMap<String, Object>();
+    nestedMap.put("date", date);
+    nestedMap.put("time", time);
+    nestedMap.put("timestamp", timestamp);
+
+    expectedRecord.put("`time_map`", nestedMap);
+  }
+
+  /**
+   * Test reading of from the parquet file that contains nested time, date, and
+   * timestamp
+   */
+  @Test
+  public void testNested() throws Exception {
+    String query = String.format("select * from %s limit 1", DATAFILE);
+    testBuilder().sqlQuery(query).ordered().baselineRecords(Arrays.asList(expectedRecord)).build().run();
+  }
+
+  /**
+   * Test timeofday() function.
+   */
+  @Test
+  public void testTimeOfDay() throws Exception {
+    ZonedDateTime now = ZonedDateTime.now();
+    // check the time zone
+    testBuilder().sqlQuery("select substr(timeofday(),25) as n from (values(1))").ordered().baselineColumns("n")
+        .baselineValues(DateUtility.formatTimeStampTZ.format(now).substring(24)).build().run();
+  }
+
+  /**
+   * Test the textual display to make sure it is consistent with actual JSON
+   * output
+   */
+  @Test
+  public void testNestedDateTimePrint() throws Exception {
+    List<QueryDataBatch> resultList = testSqlWithResults(String.format("select * from %s limit 1", DATAFILE));
+    String actual = getResultString(resultList, " | ");
+
+    final String expected = "date | time | timestamp | date_list | time_list | timestamp_list | time_map\n"
+        + "1970-01-11 | 00:00:03.600 | 2018-03-23 17:40:52.123 | [\"1970-01-11\"] | [\"00:00:03.600\"] | [\"2018-03-23 17:40:52.123\"] | {\"date\":\"1970-01-11\",\"time\":\"00:00:03.600\",\"timestamp\":\"2018-03-23 17:40:52.123\"}";
+
+    Assert.assertEquals(expected.trim(), actual.trim());
+  }
+
+  /**
+   * Test the json output is consistent as before
+   */
+  @Test
+  public void testNestedDateTimeCTASJson() throws Exception {
+    String query = String.format("select * from %s limit 1", DATAFILE);
+    String testName = "ctas_nested_datetime";
+    try {
+      test("alter session set store.format = 'json'");
+      test("alter session set store.json.extended_types = false");
+      test("use dfs.tmp");
+      test("create table " + testName + "_json as " + query);
+
+      final String readQuery = "select * from `" + testName + "_json` t1 ";
+
+      testBuilder().sqlQuery(readQuery).ordered().jsonBaselineFile("baseline_nested_datetime.json").build().run();
+    } finally {
+      test("drop table " + testName + "_json");
+      test("alter session reset store.format ");
+      test("alter session reset store.json.extended_types ");
+    }
+  }
+
+  /**
+   * Test the extended json output is consistent as before
+   */
+  @Test
+  public void testNestedDateTimeCTASExtendedJson() throws Exception {
+    String query = String.format("select * from %s limit 1", DATAFILE);
+    String testName = "ctas_nested_datetime_extended";
+    try {
+      test("alter session set store.format = 'json'");
+      test("alter session set store.json.extended_types = true");
+      test("use dfs.tmp");
+      test("create table " + testName + "_json as " + query);
+
+      final String readQuery = "select * from `" + testName + "_json` t1 ";
+
+      testBuilder().sqlQuery(readQuery).ordered().jsonBaselineFile("datetime.parquet").build().run();
+    } finally {
+      test("drop table " + testName + "_json");
+      test("alter session reset store.format ");
+      test("alter session reset store.json.extended_types ");
+    }
+  }
+
+  /**
+   * Test parquet output is consistent as before
+   */
+  @Test
+  public void testNestedDateTimeCTASParquet() throws Exception {
+    String query = String.format("select * from %s limit 1", DATAFILE);
+    String testName = "ctas_nested_datetime_extended";
+    try {
+      test("alter session set store.format = 'parquet'");
+      test("use dfs.tmp");
+      test("create table " + testName + "_parquet as " + query);
+
+      final String readQuery = "select * from `" + testName + "_parquet` t1 ";
+
+      testBuilder().sqlQuery(readQuery).ordered().jsonBaselineFile("datetime.parquet").build().run();
+    } finally {
+      test("drop table " + testName + "_parquet");
+      test("alter session reset store.format ");
+    }
+  }
+
+  /**
+   * Testing time zone change and revert
+   */
+  @Test
+  public void testTimeZoneChangeAndReverse() throws Exception {
+    long timeMillis[] = new long[] { 864000000L, 3600L, 1521826852123L };
+
+    for (int i = 0; i < timeMillis.length; i++) {
+      OffsetDateTime time1 = OffsetDateTime.ofInstant(Instant.ofEpochMilli(timeMillis[i]), ZoneOffset.UTC);
+      OffsetDateTime time2 = time1.toLocalDateTime().atZone(ZoneOffset.systemDefault()).toOffsetDateTime();
+      OffsetDateTime time3 = time2.toLocalDateTime().atOffset(ZoneOffset.UTC);
+
+      Assert.assertEquals(time1.toString(), time3.toString());
+      Assert.assertEquals(time1.toString().substring(0, 16), time2.toString().substring(0, 16));
+    }
+  }
+
+  @Test
+  public void testDateUtilityParser() {
+    LocalDateTime timestamp3 = TestBuilder.convertToLocalDateTime("1970-01-01 00:00:00.000");
+    Assert.assertEquals(timestamp3,
+        ZonedDateTime.ofInstant(Instant.parse("1970-01-01T00:00:00Z"), ZoneOffset.systemDefault()).toLocalDateTime());
+
+    String in[] = new String[] {
+        "1970-01-01",
+        "1970-01-01 20:12:32",
+        "1970-01-01 20:12:32.32",
+        "1970-01-01 20:12:32.032",
+        "1970-01-01 20:12:32.32 +0800",
+        "1970-1-01",
+        "1970-01-1 2:12:32",
+        "1970-01-01 20:12:3.32",
+        "1970-01-01 20:12:32.032",
+        "1970-01-01 20:2:32.32 +0800" };
+    for (String i : in) {
+      LocalDateTime parsed = DateUtility.parseBest(i);
+      Assert.assertNotNull(parsed);
+    }
+
+    // parse iso parser
+    String isoTimestamp[] = new String[] {
+        "2015-03-12T21:54:31.809+0530",
+        "2015-03-12T21:54:31.809Z",
+        "2015-03-12T21:54:31.809-0000",
+        "2015-03-12T21:54:31.809-0800"
+    };
+
+    for (String s : isoTimestamp) {
+      OffsetDateTime t = OffsetDateTime.parse(s, DateUtility.isoFormatTimeStamp);
+      Assert.assertNotNull(t);
+      Assert.assertNotNull(DateUtility.isoFormatTimeStamp.format(t));
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestEarlyLimit0Optimization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestEarlyLimit0Optimization.java
index 8729b69..3c7d656 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestEarlyLimit0Optimization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestEarlyLimit0Optimization.java
@@ -17,16 +17,19 @@
  */
 package org.apache.drill.exec.physical.impl.limit;
 
-import com.google.common.collect.Lists;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.List;
+
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.PlannerTest;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecConstants;
-import org.joda.time.DateTime;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.test.BaseTestQuery;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -34,8 +37,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.sql.Date;
-import java.util.List;
+import com.google.common.collect.Lists;
 
 @Category(PlannerTest.class)
 public class TestEarlyLimit0Optimization extends BaseTestQuery {
@@ -110,8 +112,8 @@ public class TestEarlyLimit0Optimization extends BaseTestQuery {
         .ordered()
         .baselineColumns("employee_id", "full_name", "position_id", "department_id", "birth_date", "hire_date",
             "salary", "fsalary", "single", "education_level", "gender")
-        .baselineValues(1, "Sheri Nowmer", 1, 1L, new DateTime(Date.valueOf("1961-08-26").getTime()),
-            new DateTime(Date.valueOf("1994-12-01").getTime()), 80000.0D, 80000.0F, true, "Graduate Degree", "F")
+        .baselineValues(1, "Sheri Nowmer", 1, 1L, LocalDate.parse("1961-08-26"),
+            LocalDateTime.parse("1994-12-01 00:00:00", DateUtility.getDateTimeFormatter()), 80000.0D, 80000.0F, true, "Graduate Degree", "F")
         .go();
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
index aa9ead3..1523913 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
@@ -26,7 +26,6 @@ import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.store.parquet.metadata.Metadata;
-import org.joda.time.DateTime;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -35,6 +34,7 @@ import org.junit.experimental.categories.Category;
 import java.io.File;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.time.LocalDate;
 
 /**
  * Tests for compatibility reading old parquet files after date corruption
@@ -144,7 +144,7 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
             .sqlQuery(query)
             .unOrdered()
             .baselineColumns("date_col")
-            .baselineValues(new DateTime(1970, 1, 1, 0, 0))
+            .baselineValues(LocalDate.of(1970, 1, 1))
             .go();
       }
     } finally {
@@ -158,8 +158,8 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
         .sqlQuery("select date_col from dfs.`%s` where length(varchar_col) = 12", VARCHAR_PARTITIONED)
         .baselineColumns("date_col")
         .unOrdered()
-        .baselineValues(new DateTime(2039, 4, 9, 0, 0))
-        .baselineValues(new DateTime(1999, 1, 8, 0, 0))
+        .baselineValues(LocalDate.of(2039, 4, 9))
+        .baselineValues(LocalDate.of(1999, 1, 8))
         .go();
   }
 
@@ -169,7 +169,7 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
         .sqlQuery("select date_col from dfs.`%s` where date_col = '1999-04-08'", DATE_PARTITIONED)
         .baselineColumns("date_col")
         .unOrdered()
-        .baselineValues(new DateTime(1999, 4, 8, 0, 0))
+        .baselineValues(LocalDate.of(1999, 4, 8))
         .go();
 
     String query = format("select date_col from dfs.`%s` where date_col > '1999-04-08'", DATE_PARTITIONED);
@@ -183,10 +183,10 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
             EXCEPTION_WHILE_PARSING_CREATED_BY_META)
         .baselineColumns("date_col")
         .unOrdered()
-        .baselineValues(new DateTime(1996, 1, 29, 0, 0))
-        .baselineValues(new DateTime(1996, 3, 1, 0, 0))
-        .baselineValues(new DateTime(1996, 3, 2, 0, 0))
-        .baselineValues(new DateTime(1997, 3, 1, 0, 0))
+        .baselineValues(LocalDate.of(1996, 1, 29))
+        .baselineValues(LocalDate.of(1996, 3, 1))
+        .baselineValues(LocalDate.of(1996, 3, 2))
+        .baselineValues(LocalDate.of(1997, 3, 1))
         .go();
   }
 
@@ -238,7 +238,7 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
               .sqlQuery(query)
               .unOrdered()
               .baselineColumns("date_col")
-              .baselineValues(new DateTime(1970, 1, 1, 0, 0))
+              .baselineValues(LocalDate.of(1970, 1, 1))
               .go();
         }
       }
@@ -265,12 +265,12 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
             PARQUET_DATE_FILE_WITH_NULL_FILLED_COLS)
         .unOrdered()
         .baselineColumns("null_dates_1", "null_dates_2", "non_existent_field", "date_col")
-        .baselineValues(null, null, null, new DateTime(1970, 1, 1, 0, 0))
-        .baselineValues(null, null, null, new DateTime(1970, 1, 2, 0, 0))
-        .baselineValues(null, null, null, new DateTime(1969, 12, 31, 0, 0))
-        .baselineValues(null, null, null, new DateTime(1969, 12, 30, 0, 0))
-        .baselineValues(null, null, null, new DateTime(1900, 1, 1, 0, 0))
-        .baselineValues(null, null, null, new DateTime(2015, 1, 1, 0, 0))
+        .baselineValues(null, null, null, LocalDate.of(1970, 1, 1))
+        .baselineValues(null, null, null, LocalDate.of(1970, 1, 2))
+        .baselineValues(null, null, null, LocalDate.of(1969, 12, 31))
+        .baselineValues(null, null, null, LocalDate.of(1969, 12, 30))
+        .baselineValues(null, null, null, LocalDate.of(1900, 1, 1))
+        .baselineValues(null, null, null, LocalDate.of(2015, 1, 1))
         .go();
   }
 
@@ -336,7 +336,7 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("date_col")
-        .baselineValues(new DateTime(1970, 1, 1, 0, 0))
+        .baselineValues(LocalDate.of(1970, 1, 1))
         .go();
   }
 
@@ -365,9 +365,9 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("date_col")
-        .baselineValues(new DateTime(1970, 1, 1, 0, 0))
-        .baselineValues(new DateTime(1970, 1, 1, 0, 0))
-        .baselineValues(new DateTime(1970, 1, 1, 0, 0))
+        .baselineValues(LocalDate.of(1970, 1, 1))
+        .baselineValues(LocalDate.of(1970, 1, 1))
+        .baselineValues(LocalDate.of(1970, 1, 1))
         .go();
   }
 
@@ -377,7 +377,7 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
         .sqlQuery("select i_rec_end_date from dfs.`%s` limit 1", CORRECT_DATES_1_6_0_PATH)
         .baselineColumns("i_rec_end_date")
         .unOrdered()
-        .baselineValues(new DateTime(2000, 10, 26, 0, 0))
+        .baselineValues(LocalDate.of(2000, 10, 26))
         .go();
   }
 
@@ -403,12 +403,12 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
 
   private void addDateBaselineValues(TestBuilder builder) {
     builder
-        .baselineValues(new DateTime(1970, 1, 1, 0, 0))
-        .baselineValues(new DateTime(1970, 1, 2, 0, 0))
-        .baselineValues(new DateTime(1969, 12, 31, 0, 0))
-        .baselineValues(new DateTime(1969, 12, 30, 0, 0))
-        .baselineValues(new DateTime(1900, 1, 1, 0, 0))
-        .baselineValues(new DateTime(2015, 1, 1, 0, 0));
+        .baselineValues(LocalDate.of(1970, 1, 1))
+        .baselineValues(LocalDate.of(1970, 1, 2))
+        .baselineValues(LocalDate.of(1969, 12, 31))
+        .baselineValues(LocalDate.of(1969, 12, 30))
+        .baselineValues(LocalDate.of(1900, 1, 1))
+        .baselineValues(LocalDate.of(2015, 1, 1));
   }
 
   /**
@@ -416,12 +416,12 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
    */
   private void addCorruptedDateBaselineValues(TestBuilder builder) {
     builder
-        .baselineValues(new DateTime(15334, 3, 17, 0, 0))
-        .baselineValues(new DateTime(15334, 3, 18, 0, 0))
-        .baselineValues(new DateTime(15334, 3, 15, 0, 0))
-        .baselineValues(new DateTime(15334, 3, 16, 0, 0))
-        .baselineValues(new DateTime(15264, 3, 16, 0, 0))
-        .baselineValues(new DateTime(15379, 3, 17, 0, 0));
+        .baselineValues(LocalDate.of(15334, 3, 17))
+        .baselineValues(LocalDate.of(15334, 3, 18))
+        .baselineValues(LocalDate.of(15334, 3, 15))
+        .baselineValues(LocalDate.of(15334, 3, 16))
+        .baselineValues(LocalDate.of(15264, 3, 16))
+        .baselineValues(LocalDate.of(15379, 3, 17));
   }
 
   private void readFilesWithUserDisabledAutoCorrection() throws Exception {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index c1dc643..9791fbd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.impl.writer;
 
 import static org.apache.drill.exec.store.parquet.ParquetRecordWriter.DRILL_VERSION_PROPERTY;
-import static org.apache.drill.test.TestBuilder.convertToLocalTimestamp;
+import static org.apache.drill.test.TestBuilder.convertToLocalDateTime;
 import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
 import static org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS;
 import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
@@ -32,7 +32,7 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.file.Paths;
-import java.sql.Date;
+import java.time.LocalDate;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -40,11 +40,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
 import org.apache.calcite.util.Pair;
-import org.apache.drill.exec.util.JsonStringArrayList;
-import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.ParquetTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.categories.UnlikelyTest;
@@ -52,6 +48,8 @@ import org.apache.drill.common.util.DrillVersionInfo;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.fn.interp.TestConstantFolding;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.util.JsonStringArrayList;
+import org.apache.drill.test.BaseTestQuery;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -61,7 +59,6 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
-import org.joda.time.DateTime;
 import org.joda.time.Period;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -71,6 +68,9 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
 @RunWith(Parameterized.class)
 @Category({SlowTest.class, ParquetTest.class})
 public class TestParquetWriter extends BaseTestQuery {
@@ -126,7 +126,7 @@ public class TestParquetWriter extends BaseTestQuery {
     allTypesSelection = Joiner.on(",").join(allTypeSelectsAndCasts);
   }
 
-  private String allTypesTable = "cp.`parquet/alltypes.json`";
+  private final String allTypesTable = "cp.`parquet/alltypes.json`";
 
   @Parameterized.Parameter
   public int repeat = 1;
@@ -624,7 +624,7 @@ public class TestParquetWriter extends BaseTestQuery {
           .unOrdered()
           .sqlQuery("SELECT * FROM dfs.tmp.`%s`", newTblName)
           .baselineColumns("id", "name", "bday")
-          .baselineValues(1, "Sheri Nowmer", new DateTime(Date.valueOf("1961-08-26").getTime()))
+          .baselineValues(1, "Sheri Nowmer", LocalDate.parse("1961-08-26"))
           .go();
     } finally {
       deleteTableIfExists(newTblName);
@@ -799,7 +799,7 @@ public class TestParquetWriter extends BaseTestQuery {
               "alter session set `%s` = true", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP)
           .ordered()
           .baselineColumns("date_value")
-          .baselineValues(new DateTime(convertToLocalTimestamp("1970-01-01 00:00:01.000")))
+          .baselineValues(convertToLocalDateTime("1970-01-01 00:00:01.000"))
           .build().run();
     } finally {
       resetSessionOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
@@ -815,7 +815,7 @@ public class TestParquetWriter extends BaseTestQuery {
                "alter session set `%s` = true", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP)
           .ordered()
           .baselineColumns("run_date")
-          .baselineValues(new DateTime(convertToLocalTimestamp("2017-12-06 16:38:43.988")))
+          .baselineValues(convertToLocalDateTime("2017-12-06 16:38:43.988"))
           .build().run();
     } finally {
       resetSessionOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
@@ -873,7 +873,7 @@ public class TestParquetWriter extends BaseTestQuery {
         .sqlQuery("SELECT convert_from(timestamp_field, 'TIMESTAMP_IMPALA')  as timestamp_field "
              + "from cp.`parquet/part1/hive_all_types.parquet` ")
         .baselineColumns("timestamp_field")
-        .baselineValues(new DateTime(convertToLocalTimestamp("2013-07-06 00:01:00")))
+        .baselineValues(convertToLocalDateTime("2013-07-06 00:01:00"))
         .baselineValues((Object)null)
         .go();
   }
@@ -1004,10 +1004,10 @@ public class TestParquetWriter extends BaseTestQuery {
           .optionSettingQueriesForTestQuery(
               "alter session set `%s` = true", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP)
           .baselineColumns("c", "d")
-          .baselineValues(new DateTime(Date.valueOf("2012-12-15").getTime()),
-              new DateTime(convertToLocalTimestamp("2016-04-24 20:06:28")))
-          .baselineValues(new DateTime(Date.valueOf("2011-07-09").getTime()),
-              new DateTime(convertToLocalTimestamp("2015-04-15 22:35:49")))
+          .baselineValues(LocalDate.parse("2012-12-15"),
+                  convertToLocalDateTime("2016-04-24 20:06:28"))
+          .baselineValues(LocalDate.parse("2011-07-09"),
+                  convertToLocalDateTime("2015-04-15 22:35:49"))
           .build()
           .run();
     } finally {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java
index b8c7275..4523b45 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java
@@ -20,8 +20,6 @@ package org.apache.drill.exec.planner.logical;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlTest;
-import org.joda.time.DateTime;
-import org.joda.time.format.DateTimeFormat;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -136,8 +134,7 @@ public class TestCaseNullableTypes extends BaseTestQuery {
 
   @Test //DRILL-5048
   public void testCaseNullableTimestamp() throws Exception {
-    DateTime date = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
-      .parseDateTime("2016-11-17 14:43:23");
+    java.time.LocalDateTime date = java.time.LocalDateTime.parse("2016-11-17 14:43:23", java.time.format.DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"));
 
     testBuilder()
       .sqlQuery("SELECT (CASE WHEN (false) THEN null ELSE CAST('2016-11-17 14:43:23' AS TIMESTAMP) END) res FROM (values(1)) foo")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
index 466a3f1..71342dc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
@@ -17,32 +17,6 @@
  */
 package org.apache.drill.exec.store.avro;
 
-import com.google.common.collect.Lists;
-import mockit.integration.junit4.JMockit;
-import org.apache.avro.specific.TestRecordWithLogicalTypes;
-import org.apache.commons.io.FileUtils;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ExecTest;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
-import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.apache.drill.exec.work.ExecErrorConstants;
-import org.apache.drill.test.BaseTestQuery;
-import org.apache.drill.test.TestBuilder;
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.exceptions.UserRemoteException;
-import org.apache.drill.exec.util.JsonStringHashMap;
-import org.joda.time.DateTime;
-import org.joda.time.LocalDate;
-import org.joda.time.LocalTime;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import java.io.File;
-import java.math.BigDecimal;
-import java.util.List;
-import java.util.Map;
-
 import static org.apache.drill.exec.store.avro.AvroTestUtil.generateDoubleNestedSchema_NoNullValues;
 import static org.apache.drill.exec.store.avro.AvroTestUtil.generateLinkedList;
 import static org.apache.drill.exec.store.avro.AvroTestUtil.generateMapSchemaComplex_withNullValues;
@@ -60,6 +34,34 @@ import static org.apache.drill.exec.store.avro.AvroTestUtil.generateUnionSchema_
 import static org.apache.drill.exec.store.avro.AvroTestUtil.write;
 import static org.apache.drill.test.TestBuilder.listOf;
 
+import java.io.File;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.avro.specific.TestRecordWithLogicalTypes;
+import org.apache.commons.io.FileUtils;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.exceptions.UserRemoteException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ExecTest;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.util.JsonStringHashMap;
+import org.apache.drill.exec.work.ExecErrorConstants;
+import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.test.TestBuilder;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import com.google.common.collect.Lists;
+
+import mockit.integration.junit4.JMockit;
+
 /**
  * Unit tests for Avro record reader.
  */
@@ -309,9 +311,12 @@ public class AvroFormatTest extends BaseTestQuery {
   @Test
   public void testAvroTableWithLogicalTypesDecimal() throws Exception {
     ExecTest.mockUtcDateTimeZone();
-    DateTime date = DateUtility.getDateTimeFormatter().parseDateTime("2018-02-03");
-    DateTime time = DateUtility.getTimeFormatter().parseDateTime("19:25:03");
-    DateTime timestamp = DateUtility.getDateTimeFormatter().parseDateTime("2018-02-03 19:25:03");
+    LocalDate date = DateUtility.parseLocalDate("2018-02-03");
+    LocalTime time = DateUtility.parseLocalTime("19:25:03.0");
+    LocalDateTime timestamp = DateUtility.parseLocalDateTime("2018-02-03 19:25:03.0");
+
+    // Avro uses joda package
+    org.joda.time.DateTime jodaDateTime = org.joda.time.DateTime.parse("2018-02-03T19:25:03");
     BigDecimal bigDecimal = new BigDecimal("123.45");
 
     TestRecordWithLogicalTypes record = new TestRecordWithLogicalTypes(
@@ -321,9 +326,9 @@ public class AvroFormatTest extends BaseTestQuery {
         3.14F,
         3019.34,
         "abc",
-        timestamp.toLocalDate(),
-        timestamp.toLocalTime(),
-        timestamp,
+        jodaDateTime.toLocalDate(),
+        jodaDateTime.toLocalTime(),
+        jodaDateTime,
         bigDecimal
     );
 
@@ -348,9 +353,9 @@ public class AvroFormatTest extends BaseTestQuery {
         3.14F,
         3019.34,
         "abc",
-        LocalDate.now(),
-        LocalTime.now(),
-        DateTime.now(),
+        org.joda.time.LocalDate.now(),
+        org.joda.time.LocalTime.now(),
+        org.joda.time.DateTime.now(),
         new BigDecimal("123.45")
     );
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
index dd3f6f3..256f1ee 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.time.ZoneOffset;
 import java.util.Arrays;
 
 import org.apache.drill.exec.memory.RootAllocator;
@@ -84,7 +85,7 @@ public class TestBsonRecordReader {
     writer.reset();
     bsonReader.write(writer, new BsonDocumentReader(bsonDoc));
     SingleMapReaderImpl mapReader = (SingleMapReaderImpl) writer.getMapVector().getReader();
-    assertEquals(1000l, mapReader.reader("ts").readDateTime().getMillis());
+    assertEquals(1000000l, mapReader.reader("ts").readLocalDateTime().atZone(ZoneOffset.systemDefault()).toInstant().toEpochMilli());
   }
 
   @Test
@@ -207,7 +208,7 @@ public class TestBsonRecordReader {
     writer.reset();
     bsonReader.write(writer, new BsonDocumentReader(bsonDoc));
     SingleMapReaderImpl mapReader = (SingleMapReaderImpl) writer.getMapVector().getReader();
-    assertEquals(5262729712L, mapReader.reader("dateTimeKey").readDateTime().getMillis());
+    assertEquals(5262729712L, mapReader.reader("dateTimeKey").readLocalDateTime().atZone(ZoneOffset.systemDefault()).toInstant().toEpochMilli());
   }
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
index a7653b8..c147980 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
@@ -17,17 +17,19 @@
  */
 package org.apache.drill.exec.store.parquet;
 
-import org.apache.drill.exec.util.JsonStringArrayList;
-import org.apache.drill.test.BaseTestQuery;
-import org.joda.time.DateTime;
-import org.joda.time.DateTimeZone;
-import org.joda.time.Period;
-import org.junit.Test;
+import static org.apache.drill.test.TestBuilder.mapOf;
 
 import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
 import java.util.Arrays;
 
-import static org.apache.drill.test.TestBuilder.mapOf;
+import org.apache.drill.exec.util.JsonStringArrayList;
+import org.apache.drill.test.BaseTestQuery;
+import org.joda.time.Period;
+import org.junit.Test;
 
 public class TestParquetComplex extends BaseTestQuery {
 
@@ -276,21 +278,21 @@ public class TestParquetComplex extends BaseTestQuery {
         .ordered()
         .baselineColumns(columns)
         .baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345, 1234567, 123, 1234, 1234567,
-            1234567890123456L, 1234567890123456L, 1234567890123456L, new DateTime("5350-02-17"),
-            new DateTime(1234567, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1973-11-29T21:33:09.012"),
+            1234567890123456L, 1234567890123456L, 1234567890123456L, LocalDate.parse("5350-02-17"),
+            LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1973-11-29T21:33:09.012"),
             new Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
             bytes12)
         .baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767, 2147483647, 255, 65535, -1,
-            9223372036854775807L, 9223372036854775807L, -1L, new DateTime("1969-12-31"),
-            new DateTime(0xFFFFFFFF, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("2038-01-19T03:14:07.999"),
+            9223372036854775807L, 9223372036854775807L, -1L, LocalDate.parse("1969-12-31"),
+            LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("2038-01-19T03:14:07.999"),
             new Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
             bytesOnes)
         .baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128, -32768, -2147483648, 0, 0, 0,
-            -9223372036854775808L, -9223372036854775808L, 0L, new DateTime("1970-01-01"),
-            new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
+            -9223372036854775808L, -9223372036854775808L, 0L, LocalDate.parse("1970-01-01"),
+            LocalDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
         .build()
         .run();
   }
@@ -350,21 +352,21 @@ public class TestParquetComplex extends BaseTestQuery {
         .ordered()
         .baselineColumns(columns)
         .baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345, 1234567, 123, 1234, 1234567,
-            1234567890123456L, 1234567890123456L, 1234567890123456L, new DateTime("5350-02-17"),
-            new DateTime(1234567, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1973-11-29T21:33:09.012"),
+            1234567890123456L, 1234567890123456L, 1234567890123456L, LocalDate.parse("5350-02-17"),
+            LocalDateTime.ofInstant(Instant.ofEpochMilli(1234567), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1973-11-29T21:33:09.012"),
             new Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
             bytes12)
         .baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767, 2147483647, 255, 65535, -1,
-            9223372036854775807L, 9223372036854775807L, -1L, new DateTime("1969-12-31"),
-            new DateTime(0xFFFFFFFF, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("2038-01-19T03:14:07.999"),
+            9223372036854775807L, 9223372036854775807L, -1L, LocalDate.parse("1969-12-31"),
+            LocalDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("2038-01-19T03:14:07.999"),
             new Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
             bytesOnes)
         .baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128, -32768, -2147483648, 0, 0, 0,
-            -9223372036854775808L, -9223372036854775808L, 0L, new DateTime("1970-01-01"),
-            new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
+            -9223372036854775808L, -9223372036854775808L, 0L, LocalDate.parse("1970-01-01"),
+            LocalDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
         .baselineValues(4, null, null, null, null, null, null, null, null, null, null, null, null, null,
             null, null, null, null)
         .build().run();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
index 6bd8dd6..bb242ba 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet2/TestDrillParquetReader.java
@@ -17,18 +17,21 @@
  */
 package org.apache.drill.exec.store.parquet2;
 
-import org.apache.drill.test.BaseTestQuery;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.util.Arrays;
+
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.joda.time.DateTime;
-import org.joda.time.DateTimeZone;
+import org.apache.drill.test.BaseTestQuery;
 import org.joda.time.Period;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.math.BigDecimal;
-import java.util.Arrays;
-
 public class TestDrillParquetReader extends BaseTestQuery {
   // enable decimal data type
   @BeforeClass
@@ -177,21 +180,21 @@ public class TestDrillParquetReader extends BaseTestQuery {
         .ordered()
         .baselineColumns(columns)
         .baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345, 1234567, 123, 1234, 1234567,
-            1234567890123456L, 1234567890123456L, 1234567890123456L, new DateTime("5350-02-17"),
-            new DateTime(1234567, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1973-11-29T21:33:09.012"),
+            1234567890123456L, 1234567890123456L, 1234567890123456L, LocalDate.parse("5350-02-17"),
+            ZonedDateTime.ofInstant(Instant.ofEpochMilli(1234567), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1973-11-29T21:33:09.012"),
             new Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
             bytes12)
         .baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767, 2147483647, 255, 65535, -1,
-            9223372036854775807L, 9223372036854775807L, -1L, new DateTime("1969-12-31"),
-            new DateTime(0xFFFFFFFF, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("2038-01-19T03:14:07.999"),
+            9223372036854775807L, 9223372036854775807L, -1L, LocalDate.parse("1969-12-31"),
+            ZonedDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("2038-01-19T03:14:07.999"),
             new Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
             bytesOnes)
         .baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128, -32768, -2147483648, 0, 0, 0,
-            -9223372036854775808L, -9223372036854775808L, 0L, new DateTime("1970-01-01"),
-            new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
+            -9223372036854775808L, -9223372036854775808L, 0L, LocalDate.parse("1970-01-01"),
+            ZonedDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
         .build()
         .run();
   }
@@ -251,21 +254,21 @@ public class TestDrillParquetReader extends BaseTestQuery {
         .ordered()
         .baselineColumns(columns)
         .baselineValues(1, "UTF8 string1", "RANDOM_VALUE", 1234567, 123, 12345, 1234567, 123, 1234, 1234567,
-            1234567890123456L, 1234567890123456L, 1234567890123456L, new DateTime("5350-02-17"),
-            new DateTime(1234567, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1973-11-29T21:33:09.012"),
+            1234567890123456L, 1234567890123456L, 1234567890123456L, LocalDate.parse("5350-02-17"),
+            ZonedDateTime.ofInstant(Instant.ofEpochMilli(1234567), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1973-11-29T21:33:09.012"),
             new Period().plusMonths(875770417).plusDays(943142453).plusMillis(1650536505),
             bytes12)
         .baselineValues(2, "UTF8 string2", "MAX_VALUE", 2147483647, 127, 32767, 2147483647, 255, 65535, -1,
-            9223372036854775807L, 9223372036854775807L, -1L, new DateTime("1969-12-31"),
-            new DateTime(0xFFFFFFFF, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("2038-01-19T03:14:07.999"),
+            9223372036854775807L, 9223372036854775807L, -1L, LocalDate.parse("1969-12-31"),
+            ZonedDateTime.ofInstant(Instant.ofEpochMilli(0xFFFFFFFF), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("2038-01-19T03:14:07.999"),
             new Period().plusMonths(16843009).plusDays(16843009).plusMillis(16843009),
             bytesOnes)
         .baselineValues(3, "UTF8 string3", "MIN_VALUE", -2147483648, -128, -32768, -2147483648, 0, 0, 0,
-            -9223372036854775808L, -9223372036854775808L, 0L, new DateTime("1970-01-01"),
-            new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(DateTimeZone.getDefault()),
-            new DateTime("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
+            -9223372036854775808L, -9223372036854775808L, 0L, LocalDate.parse("1970-01-01"),
+            ZonedDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC).toLocalTime(),
+            LocalDateTime.parse("1970-01-01T00:00:00.0"), new Period("PT0S"), bytesZeros)
         .baselineValues(4, null, null, null, null, null, null, null, null, null, null, null, null, null,
             null, null, null, null)
         .build().run();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
index 8c419ce..c2d3f9a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
@@ -80,7 +80,7 @@ public class TestExtendedTypes extends BaseTestQuery {
               1, actualRecordCount), 1, actualRecordCount);
       List<QueryDataBatch> resultList = testSqlWithResults(String.format("select * from dfs.`%s`", originalFile));
       String actual = getResultString(resultList, ",");
-      String expected = "drill_timestamp_millies,bin,bin1\n2015-07-07T03:59:43.488,drill,drill\n";
+      String expected = "drill_timestamp_millies,bin,bin1\n2015-07-07 03:59:43.488,drill,drill\n";
       Assert.assertEquals(expected, actual);
     } finally {
       resetSessionOption(ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName());
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 696b6b2..e40f86d 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
@@ -21,6 +21,9 @@ import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.sql.Timestamp;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -30,12 +33,12 @@ import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
 import org.antlr.runtime.RecognitionException;
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.drill.test.DrillTestWrapper.TestServices;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.parser.ExprLexer;
 import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.UserProtos.PreparedStatementHandle;
@@ -44,10 +47,11 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.exec.util.JsonStringHashMap;
 import org.apache.drill.exec.util.Text;
+import org.apache.drill.test.DrillTestWrapper.TestServices;
+import org.joda.time.DateTimeZone;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
-import org.joda.time.DateTimeZone;
 
 public class TestBuilder {
 
@@ -693,4 +697,15 @@ public class TestBuilder {
     long UTCTimestamp = Timestamp.valueOf(value).getTime();
     return new Timestamp(DateTimeZone.getDefault().convertUTCToLocal(UTCTimestamp));
   }
+
+  /**
+   * Helper method for the timestamp values that depend on the local timezone
+   * @param value expected timestamp value in UTC
+   * @return LocalDateTime value for the local timezone
+   */
+  public static LocalDateTime convertToLocalDateTime(String value) {
+    OffsetDateTime utcDateTime = LocalDateTime.parse(value, DateUtility.getDateTimeFormatter()).atOffset(ZoneOffset.UTC);
+    return utcDateTime.atZoneSameInstant(ZoneOffset.systemDefault()).toLocalDateTime();
+  }
+
 }
diff --git a/exec/java-exec/src/test/resources/baseline_nested_datetime.json b/exec/java-exec/src/test/resources/baseline_nested_datetime.json
new file mode 100644
index 0000000..a055e9c
--- /dev/null
+++ b/exec/java-exec/src/test/resources/baseline_nested_datetime.json
@@ -0,0 +1,13 @@
+{
+  "date" : "1970-01-11",
+  "time" : "00:00:03.600",
+  "timestamp" : "2018-03-23 17:40:52.123",
+  "date_list" : [ "1970-01-11" ],
+  "time_list" : [ "00:00:03.600" ],
+  "timestamp_list" : [ "2018-03-23 17:40:52.123" ],
+  "time_map" : {
+    "date" : "1970-01-11",
+    "time" : "00:00:03.600",
+    "timestamp" : "2018-03-23 17:40:52.123"
+  }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/datetime.parquet b/exec/java-exec/src/test/resources/datetime.parquet
new file mode 100644
index 0000000..8e5b877
Binary files /dev/null and b/exec/java-exec/src/test/resources/datetime.parquet differ
diff --git a/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd b/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
index f6e1f1d..d4851ca 100644
--- a/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
+++ b/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
@@ -68,7 +68,7 @@
         { class: "UInt4", valueHolder: "UInt4Holder" },
         { class: "Float4", javaType: "float" , boxedType: "Float", accessorType: "double", accessorCast: "set",
           fields: [{name: "value", type: "float"}]},
-        { class: "Time", javaType: "int", friendlyType: "DateTime", accessorType: "int" },
+        { class: "Time", javaType: "int", friendlyType: "LocalTime", accessorType: "int" },
         { class: "IntervalYear", javaType: "int", friendlyType: "Period" }
         { class: "Decimal9", maxPrecisionDigits: 9, friendlyType: "BigDecimal",
           fields: [{name:"value", type:"int"}, {name: "scale", type: "int", include: false},
@@ -85,8 +85,8 @@
         { class: "BigInt"},
         { class: "UInt8" },
         { class: "Float8", javaType: "double" , boxedType: "Double", fields: [{name: "value", type: "double"}], },
-        { class: "Date", javaType: "long", friendlyType: "DateTime", accessorType: "long" },
-        { class: "TimeStamp", javaType: "long", friendlyType: "DateTime", accessorType: "long" }
+        { class: "Date", javaType: "long", friendlyType: "LocalDate", accessorType: "long" },
+        { class: "TimeStamp", javaType: "long", friendlyType: "LocalDateTime", accessorType: "long" }
         { class: "Decimal18", maxPrecisionDigits: 18, friendlyType: "BigDecimal",
           fields: [{name:"value", type:"long"}, {name: "scale", type: "int", include: false},
                    {name: "precision", type: "int", include: false}] },
diff --git a/exec/vector/src/main/codegen/includes/vv_imports.ftl b/exec/vector/src/main/codegen/includes/vv_imports.ftl
index 058621c..34fe9c5 100644
--- a/exec/vector/src/main/codegen/includes/vv_imports.ftl
+++ b/exec/vector/src/main/codegen/includes/vv_imports.ftl
@@ -65,9 +65,11 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
 
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.Instant;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 
diff --git a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
index 2ed5a3b..4e5856b 100644
--- a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
+++ b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
@@ -43,7 +43,7 @@ abstract class AbstractFieldReader extends AbstractBaseReader implements FieldRe
   }
 
   <#list ["Object", "BigDecimal", "Integer", "Long", "Boolean",
-          "Character", "DateTime", "Period", "Double", "Float",
+          "Character", "LocalDate", "LocalTime", "LocalDateTime", "Period", "Double", "Float",
           "Text", "String", "Byte", "Short", "byte[]"] as friendlyType>
   <#assign safeType=friendlyType />
   <#if safeType=="byte[]"><#assign safeType="ByteArray" /></#if>
diff --git a/exec/vector/src/main/codegen/templates/FixedValueVectors.java b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
index ddd6925..68ee33b 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -27,6 +27,7 @@
 package org.apache.drill.exec.vector;
 
 <#include "/@includes/vv_imports.ftl" />
+
 import org.apache.drill.exec.util.DecimalUtility;
 
 /**
@@ -506,17 +507,13 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     @Override
     public ${friendlyType} getObject(int index) {
-      org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
-      date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-      return date;
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(get(index)), ZoneOffset.UTC).toLocalDate();
     }
     <#elseif minor.class == "TimeStamp">
 
     @Override
     public ${friendlyType} getObject(int index) {
-      org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
-      date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-      return date;
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(get(index)), ZoneOffset.UTC);
     }
     <#elseif minor.class == "IntervalYear">
 
@@ -531,10 +528,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     <#elseif minor.class == "Time">
 
     @Override
-    public DateTime getObject(int index) {
-      org.joda.time.DateTime time = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
-      time = time.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-      return time;
+    public ${friendlyType} getObject(int index) {
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(get(index)), ZoneOffset.UTC).toLocalTime();
     }
     <#elseif minor.class == "Decimal9" || minor.class == "Decimal18">
 
diff --git a/exec/vector/src/main/codegen/templates/HolderReaderImpl.java b/exec/vector/src/main/codegen/templates/HolderReaderImpl.java
index 4b7be4f..1d17a28 100644
--- a/exec/vector/src/main/codegen/templates/HolderReaderImpl.java
+++ b/exec/vector/src/main/codegen/templates/HolderReaderImpl.java
@@ -206,6 +206,12 @@ public class ${holderMode}${name}HolderReaderImpl extends AbstractFieldReader {
 
 <#elseif minor.class == "Bit" >
       return Boolean.valueOf(holder.value != 0);
+<#elseif minor.class == "Time">
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(this.holder.value), ZoneOffset.UTC).toLocalTime();
+<#elseif minor.class == "Date">
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(this.holder.value), ZoneOffset.UTC).toLocalDate();
+<#elseif minor.class == "TimeStamp">
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(this.holder.value), ZoneOffset.UTC);
 <#else>
       ${friendlyType} value = new ${friendlyType}(this.holder.value);
       return value;
@@ -281,6 +287,12 @@ public class ${holderMode}${name}HolderReaderImpl extends AbstractFieldReader {
 
 <#elseif minor.class == "Bit" >
       return Boolean.valueOf(holder.value != 0);
+<#elseif minor.class == "Time">
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(this.holder.value), ZoneOffset.UTC).toLocalTime();
+<#elseif minor.class == "Date">
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(this.holder.value), ZoneOffset.UTC).toLocalDate();
+<#elseif minor.class == "TimeStamp">
+      return LocalDateTime.ofInstant(Instant.ofEpochMilli(this.holder.value), ZoneOffset.UTC);
 <#else>
       ${friendlyType} value = new ${friendlyType}(this.holder.value);
       return value;
diff --git a/exec/vector/src/main/codegen/templates/NullReader.java b/exec/vector/src/main/codegen/templates/NullReader.java
index 4d867ba..28d48b7 100644
--- a/exec/vector/src/main/codegen/templates/NullReader.java
+++ b/exec/vector/src/main/codegen/templates/NullReader.java
@@ -123,7 +123,7 @@ public class NullReader extends AbstractBaseReader implements FieldReader {
   }
   
   <#list ["Object", "BigDecimal", "Integer", "Long", "Boolean", 
-          "Character", "DateTime", "Period", "Double", "Float",
+          "Character", "LocalDate", "LocalTime", "LocalDateTime", "Period", "Double", "Float",
           "Text", "String", "Byte", "Short", "byte[]"] as friendlyType>
   <#assign safeType=friendlyType />
   <#if safeType=="byte[]"><#assign safeType="ByteArray" /></#if>
diff --git a/exec/vector/src/main/codegen/templates/UnionReader.java b/exec/vector/src/main/codegen/templates/UnionReader.java
index 54276f5..84a2327 100644
--- a/exec/vector/src/main/codegen/templates/UnionReader.java
+++ b/exec/vector/src/main/codegen/templates/UnionReader.java
@@ -124,7 +124,7 @@ public class UnionReader extends AbstractFieldReader {
   }
 
   <#list ["Object", "BigDecimal", "Integer", "Long", "Boolean",
-          "Character", "DateTime", "Period", "Double", "Float",
+          "Character", "LocalDate", "LocalTime", "LocalDateTime", "Period", "Double", "Float",
           "Text", "String", "Byte", "Short", "byte[]"] as friendlyType>
   <#assign safeType=friendlyType />
   <#if safeType=="byte[]"><#assign safeType="ByteArray" /></#if>
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
index a52c95a..21a4352 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
@@ -17,10 +17,15 @@
  */
 package org.apache.drill.exec.expr.fn.impl;
 
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.DateTimeFormatterBuilder;
-import org.joda.time.format.DateTimeParser;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
 
 import com.carrotsearch.hppc.ObjectIntHashMap;
 
@@ -621,10 +626,14 @@ public class DateUtility {
     }
   }
 
-  public static final DateTimeFormatter formatDate        = DateTimeFormat.forPattern("yyyy-MM-dd");
-  public static final DateTimeFormatter formatTimeStamp    = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS");
-  public static final DateTimeFormatter formatTimeStampTZ = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS ZZZ");
-  public static final DateTimeFormatter formatTime        = DateTimeFormat.forPattern("HH:mm:ss.SSS");
+  public static final DateTimeFormatter formatDate        = buildFormatter("yyyy-MM-dd");
+  public static final DateTimeFormatter formatTimeStamp   = buildFormatter("yyyy-MM-dd HH:mm:ss.SSS");
+  public static final DateTimeFormatter formatTimeStampTZ = buildFormatter("yyyy-MM-dd HH:mm:ss.SSS VV");
+  public static final DateTimeFormatter formatTime        = buildFormatter("HH:mm:ss.SSS");
+
+  public static final DateTimeFormatter isoFormatDate     = formatDate;
+  public static final DateTimeFormatter isoFormatTimeStamp= buildFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSXX");
+  public static final DateTimeFormatter isoFormatTime     = buildFormatter("HH:mm:ss.SSSXX");
 
   public static DateTimeFormatter dateTimeTZFormat = null;
   public static DateTimeFormatter timeFormat = null;
@@ -639,29 +648,95 @@ public class DateUtility {
     return timezoneList[index];
   }
 
+  /**
+   * Parse given string into a LocalDate
+   */
+  public static LocalDate parseLocalDate(final String value) {
+    return LocalDate.parse(value, formatDate);
+  }
+
+  /**
+   * Parse given string into a LocalTime
+   */
+  public static LocalTime parseLocalTime(final String value) {
+    return LocalTime.parse(value, formatTime);
+  }
+
+  /**
+   * Parse the given string into a LocalDateTime.
+   */
+  public static LocalDateTime parseLocalDateTime(final String value) {
+    return LocalDateTime.parse(value, formatTimeStamp);
+  }
+
   // Returns the date time formatter used to parse date strings
   public static DateTimeFormatter getDateTimeFormatter() {
 
     if (dateTimeTZFormat == null) {
-      DateTimeFormatter dateFormatter = DateTimeFormat.forPattern("yyyy-MM-dd");
-      DateTimeParser optionalTime = DateTimeFormat.forPattern(" HH:mm:ss").getParser();
-      DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
-      DateTimeParser optionalZone = DateTimeFormat.forPattern(" ZZZ").getParser();
+      DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd");
+      DateTimeFormatter optionalTime = DateTimeFormatter.ofPattern(" HH:mm:ss");
+      DateTimeFormatter optionalSec = DateTimeFormatter.ofPattern(".SSS");
+      DateTimeFormatter optionalZone = DateTimeFormatter.ofPattern(" ZZZ");
 
-      dateTimeTZFormat = new DateTimeFormatterBuilder().append(dateFormatter).appendOptional(optionalTime).appendOptional(optionalSec).appendOptional(optionalZone).toFormatter();
+      dateTimeTZFormat = new DateTimeFormatterBuilder().parseLenient()
+          .append(dateFormatter)
+          .appendOptional(optionalTime)
+          .appendOptional(optionalSec)
+          .appendOptional(optionalZone)
+          .toFormatter();
     }
 
     return dateTimeTZFormat;
   }
 
+  /**
+   * Best effort parsing of the given value
+   */
+  public static LocalDateTime parseBest(String value) {
+    TemporalAccessor parsed = getDateTimeFormatter().parse(value);
+    LocalDate datePart;
+    LocalTime timePart;
+    ZoneOffset zoneOffset;
+
+    long epochDay = 0, nanoSeconds = 0;
+    int offsetSeconds = 0;
+
+    // get different parsed parts
+    if (parsed.isSupported(ChronoField.EPOCH_DAY)) {
+      epochDay = parsed.getLong(ChronoField.EPOCH_DAY);
+    }
+    if (parsed.isSupported(ChronoField.NANO_OF_DAY)) {
+      nanoSeconds = parsed.getLong(ChronoField.NANO_OF_DAY);
+    }
+    if (parsed.isSupported(ChronoField.OFFSET_SECONDS)) {
+      offsetSeconds = parsed.get(ChronoField.OFFSET_SECONDS);
+    }
+
+    zoneOffset = ZoneOffset.ofTotalSeconds(offsetSeconds);
+    datePart = LocalDate.ofEpochDay(epochDay);
+    timePart = LocalTime.ofNanoOfDay(nanoSeconds);
+
+    return OffsetDateTime.of(datePart, timePart, zoneOffset).toLocalDateTime();
+  }
+
   // Returns time formatter used to parse time strings
   public static DateTimeFormatter getTimeFormatter() {
     if (timeFormat == null) {
-      DateTimeFormatter timeFormatter = DateTimeFormat.forPattern("HH:mm:ss");
-      DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
-      timeFormat = new DateTimeFormatterBuilder().append(timeFormatter).appendOptional(optionalSec).toFormatter();
+      DateTimeFormatter timeFormatter = DateTimeFormatter.ofPattern("HH:mm:ss");
+      DateTimeFormatter optionalSec = DateTimeFormatter.ofPattern(".SSS");
+      timeFormat = new DateTimeFormatterBuilder().parseLenient()
+          .append(timeFormatter)
+          .appendOptional(optionalSec)
+          .toFormatter();
     }
     return timeFormat;
   }
 
+  // return a formatter that is lenient in its parsing of fields.  e.g.
+  // if the month specification is "MM", a lenient version of the formatter
+  // will accept a single digit month number as well as the 2-digit month
+  // number.
+  public static DateTimeFormatter buildFormatter(String pattern) {
+    return new DateTimeFormatterBuilder().parseLenient().append(DateTimeFormatter.ofPattern(pattern)).toFormatter();
+  }
 }
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
index 216c5cc..695befd 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
@@ -29,6 +29,7 @@ public class JsonStringArrayList<E> extends ArrayList<E> {
 
   static {
     mapper = new ObjectMapper();
+    mapper.registerModule(SerializationModule.getModule());
   }
 
   @Override
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
index e4de6d8..f4352d1 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
@@ -34,6 +34,7 @@ public class JsonStringHashMap<K, V> extends LinkedHashMap<K, V> {
 
   static {
     mapper = new ObjectMapper();
+    mapper.registerModule(SerializationModule.getModule());
   }
 
   @Override
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/SerializationModule.java b/exec/vector/src/main/java/org/apache/drill/exec/util/SerializationModule.java
new file mode 100644
index 0000000..04f4144
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/SerializationModule.java
@@ -0,0 +1,75 @@
+/*
+ * 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.exec.util;
+
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+
+/**
+ * This helper class holds any custom Jackson serializers used when outputing
+ * the data in JSON format.
+ */
+public class SerializationModule {
+
+  // copied from DateUtility. Added here for inclusion into drill-jdbc-all
+  public static final DateTimeFormatter formatDate      = DateTimeFormatter.ofPattern("uuuu-MM-dd")
+      .withZone(ZoneOffset.UTC);
+  public static final DateTimeFormatter formatTimeStamp = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss.SSS")
+      .withZone(ZoneOffset.UTC);
+  public static final DateTimeFormatter formatTime      = DateTimeFormatter.ofPattern("HH:mm:ss.SSS")
+      .withZone(ZoneOffset.UTC);
+
+  public static final SimpleModule drillModule = new SimpleModule("DrillModule");
+
+  static {
+    drillModule.addSerializer(LocalTime.class, new JsonSerializer<LocalTime>() {
+      @Override
+      public void serialize(LocalTime value, JsonGenerator gen, SerializerProvider serializers) throws IOException, JsonProcessingException {
+        gen.writeString(formatTime.format(value));
+      }
+    });
+
+    drillModule.addSerializer(LocalDate.class, new JsonSerializer<LocalDate>() {
+      @Override
+      public void serialize(LocalDate value, JsonGenerator gen, SerializerProvider serializers) throws IOException, JsonProcessingException {
+        gen.writeString(formatDate.format(value));
+      }
+    });
+
+    drillModule.addSerializer(LocalDateTime.class, new JsonSerializer<LocalDateTime>() {
+      @Override
+      public void serialize(LocalDateTime value, JsonGenerator gen, SerializerProvider serializers) throws IOException, JsonProcessingException {
+        gen.writeString(formatTimeStamp.format(value));
+      }
+    });
+  }
+
+  public static final SimpleModule getModule() {
+    return drillModule;
+  }
+}

-- 
To stop receiving notification emails like this one, please contact
amansinha@apache.org.