You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by za...@apache.org on 2023/10/06 08:56:46 UTC

[hive] branch master updated: HIVE-27760: WHERE condition on DATE type partitioning column leads to wrong results (Dayakar M reviewed by Stamatis Zampetakis)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 1e919dd9816 HIVE-27760: WHERE condition on DATE type partitioning column leads to wrong results (Dayakar M reviewed by Stamatis Zampetakis)
1e919dd9816 is described below

commit 1e919dd98169212ab10d267de0dc25bbbdf96b1e
Author: mdayakar <md...@cloudera.com>
AuthorDate: Fri Sep 29 22:36:01 2023 +0530

    HIVE-27760: WHERE condition on DATE type partitioning column leads to wrong results (Dayakar M reviewed by Stamatis Zampetakis)
    
    During partitioning pruning a DATE literal follows certain transformations that are timezone sensitive.
    
    For timezones ahead of GMT (e.g., Asia/Hong Kong) the literal is shifted by one day leading to wrong results.
    
    (2023-01-01) String -> [DateTimeFormatter] -> LocalDate -> java.sql.Date -> [SimpleDateFormat] -> String (2022-12-31)
    
    The problem was introduced by HIVE-27373, which brought DateTimeFormatter and LocalDate to the mix.
    The use of different formatters and Date object representations made parsing timezone sensitive and led to the bug.
    
    1. Add new methods in MetaStoreUtils to Encapsulate Date, Timestamp conversions from/to string.
    2. Replace SimpleDateFormat with DateTimeFormatter to keep the parsing/formatting behavior properly aligned.
    3. Refactor MetaStoreDirectSql and PartFilterVisitor to use the same conversion methods in MetaStoreUtils.
    4. Add unit tests for the new methods in MetaStoreUtils and qtests for the wrong result problem.
    
    Close apache/hive#4760
---
 .../hadoop/hive/ql/parse/BaseSemanticAnalyzer.java |   7 +-
 .../date_timestamp_partition_filter.q              |  14 ++
 .../llap/date_timestamp_partition_filter.q.out     | 214 +++++++++++++++++++++
 .../hive/metastore/utils/MetaStoreUtils.java       |  74 +++++--
 .../hadoop/hive/metastore/MetaStoreDirectSql.java  |  14 +-
 .../hive/metastore/parser/ExpressionTree.java      |   6 +-
 .../hive/metastore/parser/PartFilterVisitor.java   |  20 +-
 .../hive/metastore/utils/TestMetaStoreUtils.java   | 102 ++++++++++
 8 files changed, 398 insertions(+), 53 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 0989c412dfa..bc36832b6d1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hive.ql.parse;
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
-import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -1804,9 +1803,9 @@ public abstract class BaseSemanticAnalyzer {
       throw new SemanticException("Unexpected date type " + colValue.getClass());
     }
     try {
-      return MetaStoreUtils.PARTITION_DATE_FORMAT.get().format(
-          MetaStoreUtils.PARTITION_DATE_FORMAT.get().parse(value.toString()));
-    } catch (ParseException e) {
+      return MetaStoreUtils.convertDateToString(
+          MetaStoreUtils.convertStringToDate(value.toString()));
+    } catch (Exception e) {
       throw new SemanticException(e);
     }
   }
diff --git a/ql/src/test/queries/clientpositive/date_timestamp_partition_filter.q b/ql/src/test/queries/clientpositive/date_timestamp_partition_filter.q
new file mode 100644
index 00000000000..31328aebca8
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/date_timestamp_partition_filter.q
@@ -0,0 +1,14 @@
+--! qt:timezone:Asia/Hong_Kong
+
+CREATE EXTERNAL TABLE testpd(col1 string, col2 String) PARTITIONED BY(PartitionDate DATE) STORED AS ORC;
+INSERT into testpd(PartitionDate, col1, col2) VALUES('2023-01-01','Value11','Value12');
+INSERT into testpd(PartitionDate, col1, col2) VALUES('2023-01-02','Value21','Value22');
+explain extended select * from testpd where PartitionDate = '2023-01-01';
+select * from testpd where PartitionDate = '2023-01-01';
+
+
+CREATE EXTERNAL TABLE testpt(col1 string, col2 String) PARTITIONED BY(PartitionTimestamp TIMESTAMP) STORED AS ORC;
+INSERT into testpt(PartitionTimestamp, col1, col2) VALUES('2023-01-01 10:20:30','Value11','Value12');
+INSERT into testpt(PartitionTimestamp, col1, col2) VALUES('2023-01-02 20:30:40','Value21','Value22');
+explain extended select * from testpt where PartitionTimestamp = '2023-01-01 10:20:30';
+select * from testpt where PartitionTimestamp = '2023-01-01 10:20:30';
diff --git a/ql/src/test/results/clientpositive/llap/date_timestamp_partition_filter.q.out b/ql/src/test/results/clientpositive/llap/date_timestamp_partition_filter.q.out
new file mode 100644
index 00000000000..a676d982e37
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/date_timestamp_partition_filter.q.out
@@ -0,0 +1,214 @@
+PREHOOK: query: CREATE EXTERNAL TABLE testpd(col1 string, col2 String) PARTITIONED BY(PartitionDate DATE) STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testpd
+POSTHOOK: query: CREATE EXTERNAL TABLE testpd(col1 string, col2 String) PARTITIONED BY(PartitionDate DATE) STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testpd
+PREHOOK: query: INSERT into testpd(PartitionDate, col1, col2) VALUES('2023-01-01','Value11','Value12')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@testpd
+POSTHOOK: query: INSERT into testpd(PartitionDate, col1, col2) VALUES('2023-01-01','Value11','Value12')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@testpd
+POSTHOOK: Output: default@testpd@partitiondate=2023-01-01
+POSTHOOK: Lineage: testpd PARTITION(partitiondate=2023-01-01).col1 SCRIPT []
+POSTHOOK: Lineage: testpd PARTITION(partitiondate=2023-01-01).col2 SCRIPT []
+PREHOOK: query: INSERT into testpd(PartitionDate, col1, col2) VALUES('2023-01-02','Value21','Value22')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@testpd
+POSTHOOK: query: INSERT into testpd(PartitionDate, col1, col2) VALUES('2023-01-02','Value21','Value22')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@testpd
+POSTHOOK: Output: default@testpd@partitiondate=2023-01-02
+POSTHOOK: Lineage: testpd PARTITION(partitiondate=2023-01-02).col1 SCRIPT []
+POSTHOOK: Lineage: testpd PARTITION(partitiondate=2023-01-02).col2 SCRIPT []
+PREHOOK: query: explain extended select * from testpd where PartitionDate = '2023-01-01'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testpd
+PREHOOK: Input: default@testpd@partitiondate=2023-01-01
+#### A masked pattern was here ####
+POSTHOOK: query: explain extended select * from testpd where PartitionDate = '2023-01-01'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testpd
+POSTHOOK: Input: default@testpd@partitiondate=2023-01-01
+#### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `col1`, `col2`, CAST(DATE '2023-01-01' AS DATE) AS `partitiondate`
+FROM `default`.`testpd`
+WHERE `partitiondate` = DATE '2023-01-01'
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Partition Description:
+          Partition
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            partition values:
+              partitiondate 2023-01-01
+            properties:
+              column.name.delimiter ,
+              columns col1,col2
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.testpd
+              partition_columns partitiondate
+              partition_columns.types date
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                EXTERNAL TRUE
+                bucketing_version 2
+                column.name.delimiter ,
+                columns col1,col2
+                columns.comments 
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.testpd
+                partition_columns partitiondate
+                partition_columns.types date
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.testpd
+            name: default.testpd
+      Processor Tree:
+        TableScan
+          alias: testpd
+          filterExpr: (partitiondate = DATE'2023-01-01') (type: boolean)
+          GatherStats: false
+          Select Operator
+            expressions: col1 (type: string), col2 (type: string), DATE'2023-01-01' (type: date)
+            outputColumnNames: _col0, _col1, _col2
+            ListSink
+
+PREHOOK: query: select * from testpd where PartitionDate = '2023-01-01'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testpd
+PREHOOK: Input: default@testpd@partitiondate=2023-01-01
+#### A masked pattern was here ####
+POSTHOOK: query: select * from testpd where PartitionDate = '2023-01-01'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testpd
+POSTHOOK: Input: default@testpd@partitiondate=2023-01-01
+#### A masked pattern was here ####
+Value11	Value12	2023-01-01
+PREHOOK: query: CREATE EXTERNAL TABLE testpt(col1 string, col2 String) PARTITIONED BY(PartitionTimestamp TIMESTAMP) STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testpt
+POSTHOOK: query: CREATE EXTERNAL TABLE testpt(col1 string, col2 String) PARTITIONED BY(PartitionTimestamp TIMESTAMP) STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testpt
+PREHOOK: query: INSERT into testpt(PartitionTimestamp, col1, col2) VALUES('2023-01-01 10:20:30','Value11','Value12')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@testpt
+POSTHOOK: query: INSERT into testpt(PartitionTimestamp, col1, col2) VALUES('2023-01-01 10:20:30','Value11','Value12')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@testpt
+POSTHOOK: Output: default@testpt@partitiontimestamp=2023-01-01 10%3A20%3A30
+POSTHOOK: Lineage: testpt PARTITION(partitiontimestamp=2023-01-01 10:20:30).col1 SCRIPT []
+POSTHOOK: Lineage: testpt PARTITION(partitiontimestamp=2023-01-01 10:20:30).col2 SCRIPT []
+PREHOOK: query: INSERT into testpt(PartitionTimestamp, col1, col2) VALUES('2023-01-02 20:30:40','Value21','Value22')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@testpt
+POSTHOOK: query: INSERT into testpt(PartitionTimestamp, col1, col2) VALUES('2023-01-02 20:30:40','Value21','Value22')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@testpt
+POSTHOOK: Output: default@testpt@partitiontimestamp=2023-01-02 20%3A30%3A40
+POSTHOOK: Lineage: testpt PARTITION(partitiontimestamp=2023-01-02 20:30:40).col1 SCRIPT []
+POSTHOOK: Lineage: testpt PARTITION(partitiontimestamp=2023-01-02 20:30:40).col2 SCRIPT []
+PREHOOK: query: explain extended select * from testpt where PartitionTimestamp = '2023-01-01 10:20:30'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testpt
+PREHOOK: Input: default@testpt@partitiontimestamp=2023-01-01 10%3A20%3A30
+#### A masked pattern was here ####
+POSTHOOK: query: explain extended select * from testpt where PartitionTimestamp = '2023-01-01 10:20:30'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testpt
+POSTHOOK: Input: default@testpt@partitiontimestamp=2023-01-01 10%3A20%3A30
+#### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `col1`, `col2`, CAST(TIMESTAMP '2023-01-01 10:20:30.000000000' AS TIMESTAMP) AS `partitiontimestamp`
+FROM `default`.`testpt`
+WHERE `partitiontimestamp` = TIMESTAMP '2023-01-01 10:20:30.000000000'
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Partition Description:
+          Partition
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            partition values:
+              partitiontimestamp 2023-01-01 10:20:30
+            properties:
+              column.name.delimiter ,
+              columns col1,col2
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.testpt
+              partition_columns partitiontimestamp
+              partition_columns.types timestamp
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                EXTERNAL TRUE
+                bucketing_version 2
+                column.name.delimiter ,
+                columns col1,col2
+                columns.comments 
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.testpt
+                partition_columns partitiontimestamp
+                partition_columns.types timestamp
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.testpt
+            name: default.testpt
+      Processor Tree:
+        TableScan
+          alias: testpt
+          filterExpr: (partitiontimestamp = TIMESTAMP'2023-01-01 10:20:30') (type: boolean)
+          GatherStats: false
+          Select Operator
+            expressions: col1 (type: string), col2 (type: string), TIMESTAMP'2023-01-01 10:20:30' (type: timestamp)
+            outputColumnNames: _col0, _col1, _col2
+            ListSink
+
+PREHOOK: query: select * from testpt where PartitionTimestamp = '2023-01-01 10:20:30'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testpt
+PREHOOK: Input: default@testpt@partitiontimestamp=2023-01-01 10%3A20%3A30
+#### A masked pattern was here ####
+POSTHOOK: query: select * from testpt where PartitionTimestamp = '2023-01-01 10:20:30'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testpt
+POSTHOOK: Input: default@testpt@partitiontimestamp=2023-01-01 10%3A20%3A30
+#### A masked pattern was here ####
+Value11	Value12	2023-01-01 10:20:30
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
index 9cc59c6c5b7..a95c4c1d19b 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
@@ -23,9 +23,12 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.security.AccessController;
 import java.security.PrivilegedAction;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
+import java.time.format.ResolverStyle;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -74,25 +77,54 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Joiner;
 
 public class MetaStoreUtils {
-  /** A fixed date format to be used for hive partition column values. */
-  public static final ThreadLocal<DateFormat> PARTITION_DATE_FORMAT =
-       new ThreadLocal<DateFormat>() {
-    @Override
-    protected DateFormat initialValue() {
-      DateFormat val = new SimpleDateFormat("yyyy-MM-dd");
-      val.setLenient(false); // Without this, 2020-20-20 becomes 2021-08-20.
-      val.setTimeZone(TimeZone.getTimeZone("UTC"));
-      return val;
-    }
-  };
-  public static final ThreadLocal<DateTimeFormatter> PARTITION_TIMESTAMP_FORMAT =
-      new ThreadLocal<DateTimeFormatter>() {
-        @Override
-        protected DateTimeFormatter initialValue() {
-          return DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss").
-              withZone(TimeZone.getTimeZone("UTC").toZoneId());
-        }
-  };
+
+  private static final DateTimeFormatter DATE_FORMATTER = createDateTimeFormatter("uuuu-MM-dd");
+
+  private static final DateTimeFormatter TIMESTAMP_FORMATTER = createDateTimeFormatter("uuuu-MM-dd HH:mm:ss");
+
+  private static DateTimeFormatter createDateTimeFormatter(String format) {
+    return DateTimeFormatter.ofPattern(format).withZone(TimeZone.getTimeZone("UTC").toZoneId())
+        .withResolverStyle(ResolverStyle.STRICT);
+  }
+
+  /**
+   * Converts java.sql.Date to String format date.
+   * @param date - java.sql.Date object.
+   * @return Date in string format.
+   */
+  public static String convertDateToString(Date date) {
+    return DATE_FORMATTER.format(date.toLocalDate());
+  }
+
+  /**
+   * Converts string format date to java.sql.Date.
+   * @param date Date in string format.
+   * @return java.sql.Date object.
+   */
+  public static Date convertStringToDate(String date) {
+    LocalDate val = LocalDate.parse(date, DATE_FORMATTER);
+    return java.sql.Date.valueOf(val);
+  }
+
+  /**
+   * Converts java.sql.Timestamp to string format timestamp.
+   * @param timestamp java.sql.Timestamp object.
+   * @return Timestamp in string format.
+   */
+  public static String convertTimestampToString(Timestamp timestamp) {
+    return TIMESTAMP_FORMATTER.format(timestamp.toLocalDateTime());
+  }
+
+  /**
+   * Converts timestamp string format to java.sql.Timestamp.
+   * @param timestamp Timestamp in string format.
+   * @return java.sql.Timestamp object.
+   */
+  public static Timestamp convertStringToTimestamp(String timestamp) {
+    LocalDateTime val = LocalDateTime.from(TIMESTAMP_FORMATTER.parse(timestamp));
+    return Timestamp.valueOf(val);
+  }
+
   // Indicates a type was derived from the deserializer rather than Hive's metadata.
   public static final String TYPE_FROM_DESERIALIZER = "<derived from deserializer>";
 
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index aeb1ef9f6c2..92865954bcd 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -32,11 +32,10 @@ import static org.apache.hadoop.hive.metastore.ColumnType.TINYINT_TYPE_NAME;
 import static org.apache.hadoop.hive.metastore.ColumnType.VARCHAR_TYPE_NAME;
 
 import java.sql.Connection;
+import java.sql.Date;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Timestamp;
-import java.text.ParseException;
-import java.time.LocalDateTime;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -1352,25 +1351,24 @@ class MetaStoreDirectSql {
       if (colType == FilterType.Date && valType == FilterType.String) {
         // Filter.g cannot parse a quoted date; try to parse date here too.
         try {
-          nodeValue = MetaStoreUtils.PARTITION_DATE_FORMAT.get().parse((String)nodeValue);
+          nodeValue = MetaStoreUtils.convertStringToDate((String)nodeValue);
           valType = FilterType.Date;
-        } catch (ParseException pe) { // do nothing, handled below - types will mismatch
+        } catch (Exception pe) { // do nothing, handled below - types will mismatch
         }
       }
 
       if (colType == FilterType.Timestamp && valType == FilterType.String) {
-        nodeValue = Timestamp.valueOf(LocalDateTime.from(
-            MetaStoreUtils.PARTITION_TIMESTAMP_FORMAT.get().parse((String)nodeValue)));
+        nodeValue = MetaStoreUtils.convertStringToTimestamp((String)nodeValue);
         valType = FilterType.Timestamp;
       }
 
       // We format it so we are sure we are getting the right value
       if (valType == FilterType.Date) {
         // Format
-        nodeValue = MetaStoreUtils.PARTITION_DATE_FORMAT.get().format(nodeValue);
+        nodeValue = MetaStoreUtils.convertDateToString((Date)nodeValue);
       } else if (valType == FilterType.Timestamp) {
         //format
-        nodeValue = MetaStoreUtils.PARTITION_TIMESTAMP_FORMAT.get().format(((Timestamp) nodeValue).toLocalDateTime());
+        nodeValue = MetaStoreUtils.convertTimestampToString((Timestamp) nodeValue);
       }
 
       boolean isDefaultPartition = (valType == FilterType.String) && defaultPartName.equals(nodeValue);
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java
index 28b1c683bc8..f2f91cbedfb 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hive.metastore.parser;
 
 import java.sql.Timestamp;
-import java.util.Date;
+import java.sql.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -486,9 +486,9 @@ public class ExpressionTree {
       // columns have been excluded above, so it will either compare w/string or fail.
       Object val = value;
       if (colType.equals("date") && value instanceof Date) {
-        val = MetaStoreUtils.PARTITION_DATE_FORMAT.get().format((Date)value);
+        val = MetaStoreUtils.convertDateToString((Date)value);
       } else if (colType.equals("timestamp") && value instanceof Timestamp) {
-        val = MetaStoreUtils.PARTITION_TIMESTAMP_FORMAT.get().format(((Timestamp)value).toLocalDateTime());
+        val = MetaStoreUtils.convertTimestampToString((Timestamp)value);
       }
       boolean isStringValue = val instanceof String;
       if (!isStringValue && (!isIntegralSupported || !(val instanceof Long))) {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/PartFilterVisitor.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/PartFilterVisitor.java
index f5fe429b7c2..5d68d593c83 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/PartFilterVisitor.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/PartFilterVisitor.java
@@ -19,20 +19,16 @@ package org.apache.hadoop.hive.metastore.parser;
 
 import java.sql.Date;
 import java.sql.Timestamp;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.format.DateTimeFormatter;
 import java.time.format.DateTimeParseException;
-import java.time.format.ResolverStyle;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.TimeZone;
 import java.util.stream.Collectors;
 
 import org.antlr.v4.runtime.misc.ParseCancellationException;
 import org.antlr.v4.runtime.tree.RuleNode;
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 
 import static org.apache.hadoop.hive.metastore.parser.ExpressionTree.LeafNode;
 import static org.apache.hadoop.hive.metastore.parser.ExpressionTree.LogicalOperator;
@@ -40,14 +36,6 @@ import static org.apache.hadoop.hive.metastore.parser.ExpressionTree.Operator;
 import static org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeNode;
 
 public class PartFilterVisitor extends PartitionFilterBaseVisitor<Object> {
-  private final DateTimeFormatter dateFormat = createDateTimeFormatter("uuuu-MM-dd");
-  private final DateTimeFormatter timestampFormat = createDateTimeFormatter("uuuu-MM-dd HH:mm:ss");
-
-  private DateTimeFormatter createDateTimeFormatter(String format) {
-    return DateTimeFormatter.ofPattern(format)
-            .withZone(TimeZone.getTimeZone("UTC").toZoneId())
-            .withResolverStyle(ResolverStyle.STRICT);
-  }
 
   /**
    * Override the default behavior for all visit methods. This will only return a non-null result
@@ -247,8 +235,7 @@ public class PartFilterVisitor extends PartitionFilterBaseVisitor<Object> {
     PartitionFilterParser.DateContext date = ctx.date();
     String dateValue = unquoteString(date.value.getText());
     try {
-      LocalDate localDate = LocalDate.parse(dateValue, dateFormat);
-      return Date.valueOf(localDate);
+      return MetaStoreUtils.convertStringToDate(dateValue);
     } catch (DateTimeParseException e) {
       throw new ParseCancellationException(e.getMessage());
     }
@@ -259,8 +246,7 @@ public class PartFilterVisitor extends PartitionFilterBaseVisitor<Object> {
     PartitionFilterParser.TimestampContext timestamp = ctx.timestamp();
     String timestampValue = unquoteString(timestamp.value.getText());
     try {
-      LocalDateTime val = LocalDateTime.from(timestampFormat.parse(timestampValue));
-      return Timestamp.valueOf(val);
+      return MetaStoreUtils.convertStringToTimestamp(timestampValue);
     } catch (DateTimeParseException e) {
       throw new ParseCancellationException(e.getMessage());
     }
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
new file mode 100644
index 00000000000..8632f233e04
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
@@ -0,0 +1,102 @@
+/*
+ * 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
+ * <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.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.utils;
+
+import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import java.util.TimeZone;
+
+import static org.junit.Assert.assertEquals;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreUnitTest.class)
+public class TestMetaStoreUtils {
+  private static final TimeZone DEFAULT = TimeZone.getDefault();
+  private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss");
+  private final TimeZone timezone;
+  private final String date;
+  private final String timestamp;
+
+  public TestMetaStoreUtils(String zoneId, LocalDateTime timestamp) {
+    this.timezone = TimeZone.getTimeZone(zoneId);
+    this.timestamp = timestamp.format(FORMATTER);
+    this.date = timestamp.toLocalDate().format(DateTimeFormatter.ISO_LOCAL_DATE);
+  }
+
+  @Parameterized.Parameters(name = "zoneId={0}, timestamp={1}")
+  public static Collection<Object[]> generateZoneTimestampPairs() {
+    List<Object[]> params = new ArrayList<>();
+    long minDate = LocalDate.of(0, 1, 1).atStartOfDay().toEpochSecond(ZoneOffset.UTC);
+    long maxDate = LocalDate.of(9999, 12, 31).atStartOfDay().toEpochSecond(ZoneOffset.UTC);
+    new Random(23).longs(500, minDate, maxDate).forEach(i -> {
+      LocalDateTime datetime = LocalDateTime.ofEpochSecond(i, 0, ZoneOffset.UTC);
+      for (String zone : ZoneId.SHORT_IDS.values()) {
+        params.add(new Object[] { zone, datetime });
+      }
+    });
+    return params;
+  }
+
+  @Before
+  public void setup() {
+    TimeZone.setDefault(timezone);
+  }
+
+  @Test
+  public void testDateToString() {
+    assertEquals(date, MetaStoreUtils.convertDateToString(Date.valueOf(date)));
+  }
+
+  @Test
+  public void testTimestampToString() {
+    assertEquals(timestamp, MetaStoreUtils.convertTimestampToString(Timestamp.valueOf(timestamp)));
+  }
+
+  @Test
+  public void testStringToDate() {
+    assertEquals(Date.valueOf(date), MetaStoreUtils.convertStringToDate(date));
+  }
+
+  @Test
+  public void testStringToTimestamp() {
+    assertEquals(Timestamp.valueOf(timestamp), MetaStoreUtils.convertStringToTimestamp(timestamp));
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    TimeZone.setDefault(DEFAULT);
+  }
+}