You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2017/11/13 12:07:17 UTC

[03/11] drill git commit: DRILL-5717: Let some test cases be Local or TimeZone independent.

DRILL-5717: Let some test cases be Local or TimeZone independent.

closes #904


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/17ca6181
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/17ca6181
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/17ca6181

Branch: refs/heads/master
Commit: 17ca6181ba924837d493ae789637b3d4514bd560
Parents: 496c97d
Author: weijie.tong <we...@alipay.com>
Authored: Sun Aug 20 20:07:41 2017 +0800
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Mon Nov 13 11:05:13 2017 +0200

----------------------------------------------------------------------
 .../java/org/apache/drill/exec/ExecTest.java    |  35 ++-
 .../drill/exec/fn/impl/TestCastFunctions.java   |  22 +-
 .../drill/exec/fn/impl/TestDateFunctions.java   | 226 ++++++++++---------
 .../exec/fn/impl/TestNewDateFunctions.java      |   8 +-
 .../fn/impl/testing/TestDateConversions.java    |  19 +-
 .../exec/fn/interp/TestConstantFolding.java     |  17 +-
 .../complex/writer/TestExtendedTypes.java       |  19 +-
 .../expression/fn/JodaDateValidatorTest.java    |   7 +-
 8 files changed, 200 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index 4200f89..418e4a4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -19,11 +19,13 @@ package org.apache.drill.exec;
 
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.io.Files;
+import mockit.Mock;
+import mockit.MockUp;
 import mockit.NonStrictExpectations;
-import org.apache.commons.io.FileUtils;
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
 import org.antlr.runtime.RecognitionException;
+import org.apache.commons.io.FileUtils;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.parser.ExprLexer;
@@ -35,18 +37,20 @@ import org.apache.drill.exec.metrics.DrillMetrics;
 import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
 import org.apache.drill.exec.util.GuavaPatcher;
 import org.apache.drill.test.DrillTest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.joda.time.DateTimeUtils;
+import org.joda.time.DateTimeZone;
 import org.junit.After;
 import org.junit.BeforeClass;
-
 import java.io.File;
 import java.io.IOException;
+import java.text.DateFormatSymbols;
+import java.util.Locale;
 
 
 public class ExecTest extends DrillTest {
@@ -120,4 +124,29 @@ public class ExecTest extends DrillTest {
     return ret.e;
   }
 
+  /**
+   * This utility is to mock the method DateTimeUtils.getDateFormatSymbols()
+   * to mock the current local as US.
+   */
+  public static void mockUsDateFormatSymbols() {
+    new MockUp<DateTimeUtils>() {
+      @Mock
+      public DateFormatSymbols getDateFormatSymbols(Locale locale) {
+        return new DateFormatSymbols(Locale.US);
+      }
+    };
+  }
+
+  /**
+   * This utility is to mock the method DateTimeZone.getDefault() to
+   * mock current timezone as UTC.
+   */
+  public static void mockUtcDateTimeZone() {
+    new MockUp<DateTimeZone>() {
+      @Mock
+      public DateTimeZone getDefault() {
+        return DateTimeZone.UTC;
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
----------------------------------------------------------------------
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 42a048e..4aeb396 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
@@ -19,6 +19,7 @@ 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.BaseTestQuery;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
@@ -26,11 +27,12 @@ import org.apache.drill.common.util.FileUtils;
 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.util.List;
 import java.util.Map;
 
+@RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestCastFunctions extends BaseTestQuery {
 
@@ -77,16 +79,18 @@ public class TestCastFunctions extends BaseTestQuery {
 
   @Test // DRILL-3769
   public void testToDateForTimeStamp() throws Exception {
-    final String query = "select to_date(to_timestamp(-1)) as col \n" +
-        "from (values(1))";
+    mockUtcDateTimeZone();
+
+    final String query = "select to_date(to_timestamp(-1)) as col \n"
+      + "from (values(1))";
 
     testBuilder()
-        .sqlQuery(query)
-        .ordered()
-        .baselineColumns("col")
-        .baselineValues(new DateTime(1969, 12, 31, 0, 0))
-        .build()
-        .run();
+      .sqlQuery(query)
+      .ordered()
+      .baselineColumns("col")
+      .baselineValues(new DateTime(1969, 12, 31, 0, 0))
+      .build()
+      .run();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
index a491131..622d8f4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -17,11 +17,9 @@
  */
 package org.apache.drill.exec.fn.impl;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import mockit.integration.junit4.JMockit;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.util.FileUtils;
@@ -34,127 +32,131 @@ import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
 import org.joda.time.LocalDate;
-import org.joda.time.LocalTime;
 import org.joda.time.LocalDateTime;
+import org.joda.time.LocalTime;
 import org.junit.Ignore;
 import org.junit.Test;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import java.util.List;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
+@RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestDateFunctions extends PopUnitTestBase {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDateFunctions.class);
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDateFunctions.class);
 
-    public void testCommon(String[] expectedResults, String physicalPlan, String resourceFile) throws Exception {
-        try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
 
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    Files.toString(FileUtils.getResourceAsFile(physicalPlan), Charsets.UTF_8)
-                            .replace("#{TEST_FILE}", resourceFile));
+  public void testCommon(String[] expectedResults, String physicalPlan, String resourceFile) throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
 
-            RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+        Files.toString(FileUtils.getResourceAsFile(physicalPlan), Charsets.UTF_8)
+          .replace("#{TEST_FILE}", resourceFile));
 
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
+      RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
-            int i = 0;
-            for (VectorWrapper<?> v : batchLoader) {
 
-                ValueVector.Accessor accessor = v.getValueVector().getAccessor();
-                System.out.println(accessor.getObject(0));
-                assertEquals( expectedResults[i++], accessor.getObject(0).toString());
-            }
-
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-                b.release();
-            }
-        }
-    }
-
-    @Test
-    @Ignore("relies on particular timezone")
-    public void testDateIntervalArithmetic() throws Exception {
-        String expectedResults[] = {"2009-02-23T00:00:00.000-08:00",
-                                    "2008-02-24T00:00:00.000-08:00",
-                                    "1970-01-01T13:20:33.000-08:00",
-                                    "2008-02-24T12:00:00.000-08:00",
-                                    "2009-04-23T12:00:00.000-07:00",
-                                    "2008-02-24T12:00:00.000-08:00",
-                                    "2009-04-23T12:00:00.000-07:00",
-                                    "2009-02-23T00:00:00.000-08:00",
-                                    "2008-02-24T00:00:00.000-08:00",
-                                    "1970-01-01T13:20:33.000-08:00",
-                                    "2008-02-24T12:00:00.000-08:00",
-                                    "2009-04-23T12:00:00.000-07:00",
-                                    "2008-02-24T12:00:00.000-08:00",
-                                    "2009-04-23T12:00:00.000-07:00"};
-        testCommon(expectedResults, "/functions/date/date_interval_arithmetic.json", "/test_simple_date.json");
-    }
-
-    @Test
-    public void testDateDifferenceArithmetic() throws Exception {
-
-        String[] expectedResults = {"P365D",
-                                    "P-366DT-60S",
-                                    "PT39600S"};
-        testCommon(expectedResults, "/functions/date/date_difference_arithmetic.json", "/test_simple_date.json");
-    }
-
-    @Test
-    public void testAge() throws Exception {
-        String[] expectedResults = { "P109M16DT82800S",
-                                     "P172M27D",
-                                     "P-172M-27D",
-                                     "P-39M-18DT-63573S"};
-        testCommon(expectedResults, "/functions/date/age.json", "/test_simple_date.json");
-    }
-
-    @Test
-    public void testIntervalArithmetic() throws Exception {
-
-      String expectedResults[] = {"P2Y2M",
-          "P2DT3723S",
-          "P2M",
-          "PT3723S",
-          "P28M",
-          "PT7206S",
-          "P7M",
-          "PT1801.500S",
-          "P33M18D",
-          "PT8647.200S",
-          "P6M19DT86399.999S",
-          "PT1715.714S"};
-
-        testCommon(expectedResults, "/functions/date/interval_arithmetic.json", "/test_simple_date.json");
-    }
-
-    @Test
-    public void testToChar() throws Exception {
-
-        String expectedResults[] = {(new LocalDate(2008, 2, 23)).toString("yyyy-MMM-dd"),
-                                    (new LocalTime(12, 20, 30)).toString("HH mm ss"),
-                                    (new LocalDateTime(2008, 2, 23, 12, 0, 0)).toString("yyyy MMM dd HH:mm:ss")};
-        testCommon(expectedResults, "/functions/date/to_char.json", "/test_simple_date.json");
-    }
+      int i = 0;
+      for (VectorWrapper<?> v : batchLoader) {
 
-    @Test
-    @Ignore("relies on particular time zone")
-    public void testToDateType() throws Exception {
-        String expectedResults[] = {"2008-02-23T00:00:00.000-08:00",
-                                    "1970-01-01T12:20:30.000-08:00",
-                                    "2008-02-23T12:00:00.000-08:00",
-                                    "2008-02-23T12:00:00.000-08:00"};
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
+        System.out.println(accessor.getObject(0));
+        assertEquals(expectedResults[i++], accessor.getObject(0).toString());
+      }
 
-        testCommon(expectedResults, "/functions/date/to_date_type.json", "/test_simple_date.json");
+      batchLoader.clear();
+      for (QueryDataBatch b : results) {
+        b.release();
+      }
     }
+  }
+
+  @Test
+  @Ignore("relies on particular timezone")
+  public void testDateIntervalArithmetic() throws Exception {
+    String expectedResults[] = {"2009-02-23T00:00:00.000-08:00",
+                                "2008-02-24T00:00:00.000-08:00",
+                                "1970-01-01T13:20:33.000-08:00",
+                                "2008-02-24T12:00:00.000-08:00",
+                                "2009-04-23T12:00:00.000-07:00",
+                                "2008-02-24T12:00:00.000-08:00",
+                                "2009-04-23T12:00:00.000-07:00",
+                                "2009-02-23T00:00:00.000-08:00",
+                                "2008-02-24T00:00:00.000-08:00",
+                                "1970-01-01T13:20:33.000-08:00",
+                                "2008-02-24T12:00:00.000-08:00",
+                                "2009-04-23T12:00:00.000-07:00",
+                                "2008-02-24T12:00:00.000-08:00",
+                                "2009-04-23T12:00:00.000-07:00"};
+    testCommon(expectedResults, "/functions/date/date_interval_arithmetic.json", "/test_simple_date.json");
+  }
+
+  @Test
+  public void testDateDifferenceArithmetic() throws Exception {
+
+    String[] expectedResults = {"P365D",
+                                "P-366DT-60S",
+                                "PT39600S"};
+    testCommon(expectedResults, "/functions/date/date_difference_arithmetic.json", "/test_simple_date.json");
+  }
+
+  @Test
+  public void testAge() throws Exception {
+    String[] expectedResults = {"P109M16DT82800S",
+                                "P172M27D",
+                                "P-172M-27D",
+                                "P-39M-18DT-63573S"};
+    testCommon(expectedResults, "/functions/date/age.json", "/test_simple_date.json");
+  }
+
+  @Test
+  public void testIntervalArithmetic() throws Exception {
+
+    String expectedResults[] = {"P2Y2M",
+                                "P2DT3723S",
+                                "P2M",
+                                "PT3723S",
+                                "P28M",
+                                "PT7206S",
+                                "P7M",
+                                "PT1801.500S",
+                                "P33M18D",
+                                "PT8647.200S",
+                                "P6M19DT86399.999S",
+                                "PT1715.714S"};
+
+    testCommon(expectedResults, "/functions/date/interval_arithmetic.json", "/test_simple_date.json");
+  }
+
+  @Test
+  public void testToChar() throws Exception {
+    mockUsDateFormatSymbols();
+
+    String expectedResults[] = {(new LocalDate(2008, 2, 23)).toString("yyyy-MMM-dd"),
+                                (new LocalTime(12, 20, 30)).toString("HH mm ss"),
+                                (new LocalDateTime(2008, 2, 23, 12, 0, 0)).toString("yyyy MMM dd HH:mm:ss")};
+    testCommon(expectedResults, "/functions/date/to_char.json", "/test_simple_date.json");
+  }
+
+  @Test
+  @Ignore("relies on particular time zone")
+  public void testToDateType() throws Exception {
+    String expectedResults[] = {"2008-02-23T00:00:00.000-08:00",
+                                "1970-01-01T12:20:30.000-08:00",
+                                "2008-02-23T12:00:00.000-08:00",
+                                "2008-02-23T12:00:00.000-08:00"};
+
+    testCommon(expectedResults, "/functions/date/to_date_type.json", "/test_simple_date.json");
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java
----------------------------------------------------------------------
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 49eb795..e339117 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
@@ -26,8 +26,6 @@ import org.joda.time.format.DateTimeFormatter;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.sql.Date;
-
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestNewDateFunctions extends BaseTestQuery {
   DateTime date;
@@ -55,9 +53,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(Date.valueOf("1900-01-01").getTime()))
-        .baselineValues(new DateTime(Date.valueOf("3500-01-01").getTime()))
-        .baselineValues(new DateTime(Date.valueOf("2000-12-31").getTime()))
+        .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(new Object[] {null})
         .baselineValues(new Object[] {null})
         .baselineValues(new Object[] {null})

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
index 27599c2..4da6db3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
@@ -16,6 +16,7 @@
 */
 package org.apache.drill.exec.fn.impl.testing;
 
+import mockit.integration.junit4.JMockit;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
@@ -24,15 +25,15 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.junit.runner.RunWith;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
-
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertThat;
 
+@RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestDateConversions extends BaseTestQuery {
 
@@ -104,6 +105,8 @@ public class TestDateConversions extends BaseTestQuery {
 
   @Test
   public void testJodaTime() throws Exception {
+    mockUsDateFormatSymbols();
+
     String query = String.format("SELECT to_time(time1, 'H:m:ss') = "
       + "to_time(time2, 'h:m:ssa') as col1, "
       + "to_time(time1, 'H:m:ss') = "
@@ -121,6 +124,8 @@ public class TestDateConversions extends BaseTestQuery {
 
   @Test
   public void testPostgresTime() throws Exception {
+    mockUsDateFormatSymbols();
+
     String query = String.format("SELECT sql_to_time(time1, 'HH24:MI:SS') = "
       + "sql_to_time(time2, 'HH12:MI:SSam') as col1, "
       + "sql_to_time(time1, 'HH24:MI:SS') = "
@@ -138,6 +143,8 @@ public class TestDateConversions extends BaseTestQuery {
 
   @Test
   public void testPostgresDateTime() throws Exception {
+    mockUsDateFormatSymbols();
+
     String query = String.format("SELECT sql_to_timestamp(time1, 'yyyy-DD-MMHH24:MI:SS') = "
       + "sql_to_timestamp(time2, 'DDMMyyyyHH12:MI:SSam') as col1, "
       + "sql_to_timestamp(time1, 'yyyy-DD-MMHH24:MI:SS') = "
@@ -151,11 +158,12 @@ public class TestDateConversions extends BaseTestQuery {
       .baselineValues(true, true)
       .baselineValues(false, true)
       .go();
-
   }
 
   @Test
   public void testJodaDateTime() throws Exception {
+    mockUsDateFormatSymbols();
+
     String query = String.format("SELECT to_timestamp(time1, 'yyyy-dd-MMH:m:ss') = "
       + "to_timestamp(time2, 'ddMMyyyyh:m:ssa') as col1, "
       + "to_timestamp(time1, 'yyyy-dd-MMH:m:ss') = "
@@ -173,6 +181,8 @@ public class TestDateConversions extends BaseTestQuery {
 
   @Test
   public void testJodaDateTimeNested() throws Exception {
+    mockUsDateFormatSymbols();
+
     String query = String.format("SELECT date_add(to_date(time1, concat('yyyy-dd-MM','H:m:ss')), 22)= "
       + "date_add(to_date(time2, concat('ddMMyyyy', 'h:m:ssa')), 22) as col1, "
       + "date_add(to_date(time1, concat('yyyy-dd-MM', 'H:m:ss')), 22) = "
@@ -186,11 +196,12 @@ public class TestDateConversions extends BaseTestQuery {
       .baselineValues(true, true)
       .baselineValues(false, true)
       .go();
-
   }
 
   @Test
   public void testPostgresDateTimeNested() throws Exception {
+    mockUsDateFormatSymbols();
+
     String query = String.format("SELECT date_add(sql_to_date(time1, concat('yyyy-DD-MM', 'HH24:MI:SS')), 22) = "
       + "date_add(sql_to_date(time2, concat('DDMMyyyy', 'HH12:MI:SSam')), 22) as col1, "
       + "date_add(sql_to_date(time1, concat('yyyy-DD-MM', 'HH24:MI:SS')), 10) = "

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
index 5f55d2d..206bf97 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
@@ -17,10 +17,9 @@
  ******************************************************************************/
 package org.apache.drill.exec.fn.interp;
 
-import java.io.File;
-import java.io.PrintWriter;
-import java.util.List;
-
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import mockit.integration.junit4.JMockit;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.SqlTest;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
@@ -29,10 +28,12 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import java.io.File;
+import java.io.PrintWriter;
+import java.util.List;
 
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
-
+@RunWith(JMockit.class)
 @Category(SqlTest.class)
 public class TestConstantFolding extends PlanTestBase {
 
@@ -117,6 +118,8 @@ public class TestConstantFolding extends PlanTestBase {
 
   @Test
   public void testConstantFolding_allTypes() throws Exception {
+    mockUsDateFormatSymbols();
+
     try {
       test("alter session set `store.json.all_text_mode` = true;");
       test(String.format("alter session set `%s` = true", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
----------------------------------------------------------------------
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 4d2da29..0710818 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
@@ -17,14 +17,7 @@
  */
 package org.apache.drill.exec.vector.complex.writer;
 
-import static org.junit.Assert.assertEquals;
-
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
+import mockit.integration.junit4.JMockit;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
@@ -32,12 +25,21 @@ import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.test.OperatorFixture;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import static org.junit.Assert.assertEquals;
 
+@RunWith(JMockit.class)
 public class TestExtendedTypes extends BaseTestQuery {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExtendedTypes.class);
 
   @Test
   public void checkReadWriteExtended() throws Exception {
+    mockUtcDateTimeZone();
 
     final String originalFile = "${WORKING_PATH}/src/test/resources/vector/complex/extended.json".replaceAll(
         Pattern.quote("${WORKING_PATH}"),
@@ -53,7 +55,6 @@ public class TestExtendedTypes extends BaseTestQuery {
 
       // check query of table.
       test("select * from dfs_test.tmp.`%s`", newTable);
-
       // check that original file and new file match.
       final byte[] originalData = Files.readAllBytes(Paths.get(originalFile));
       final byte[] newData = Files.readAllBytes(Paths.get(BaseTestQuery.getDfsTestTmpSchemaLocation() + '/' + newTable

http://git-wip-us.apache.org/repos/asf/drill/blob/17ca6181/logical/src/test/java/org/apache/drill/common/expression/fn/JodaDateValidatorTest.java
----------------------------------------------------------------------
diff --git a/logical/src/test/java/org/apache/drill/common/expression/fn/JodaDateValidatorTest.java b/logical/src/test/java/org/apache/drill/common/expression/fn/JodaDateValidatorTest.java
index 8398bcf..2bd4fd4 100644
--- a/logical/src/test/java/org/apache/drill/common/expression/fn/JodaDateValidatorTest.java
+++ b/logical/src/test/java/org/apache/drill/common/expression/fn/JodaDateValidatorTest.java
@@ -23,9 +23,8 @@ import org.joda.time.format.DateTimeFormatter;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
+import java.util.Locale;
 import java.util.Map;
-
 import static org.apache.drill.common.expression.fn.JodaDateValidator.toJodaFormat;
 import static org.joda.time.DateTime.parse;
 import static org.joda.time.format.DateTimeFormat.forPattern;
@@ -125,7 +124,7 @@ public class JodaDateValidatorTest {
     DateTime date = parseDateFromPostgres(hours + ":" + minutes + ":" + seconds + " am", "hh12:mi:ss am");
     Assert.assertTrue(date.getHourOfDay() == hours &&
                         date.getMinuteOfHour() == minutes &&
-                        date.getSecondOfMinute() == Integer.parseInt(seconds));
+                          date.getSecondOfMinute() == Integer.parseInt(seconds));
   }
 
   @Test
@@ -197,7 +196,7 @@ public class JodaDateValidatorTest {
 
   private DateTime parseDateFromPostgres(String date, String pattern) {
     String jodaFormat = toJodaFormat(pattern);
-    DateTimeFormatter format = forPattern(jodaFormat);
+    DateTimeFormatter format = forPattern(jodaFormat).withLocale(Locale.US);
     return parse(date, format).withZoneRetainFields(DateTimeZone.UTC);
   }
 }