[ https://issues.apache.org/jira/browse/HIVE-8102?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14133985#comment-14133985 ]
Eli Acherkan commented on HIVE-8102: ------------------------------------ The following test fails when running in Asia/Jerusalem timezone: Date originalDate = new Date(114, 2, 28); // March 28th 2014 - DST begins on this day at 02:00. DateWritable dateWritable = new DateWritable(originalDate); assertEquals(originalDate, dateWritable.get()); // Assertion fails because dateWritable.get() returns 2014-03-27 23:00:00 IST. In order to be able to run this unit test in any timezone, we explicitly set the timezone and run it in a separate thread, so that the thread local member DataWritable.LOCAL_TIMEZONE is initialized with the correct one: public void testDaylightSavingsTime() throws InterruptedException, ExecutionException { TimeZone previousDefault = TimeZone.getDefault(); TimeZone.setDefault(TimeZone.getTimeZone("Asia/Jerusalem")); ExecutorService threadPool = Executors.newFixedThreadPool(1); try { Future<Boolean> future = threadPool.submit(new Callable<Boolean>() { @Override public Boolean call() throws Exception { Date originalDate = new Date(114, 2, 28); // March 28th 2014 - DST begins on this day at 02:00. DateWritable dateWritable = new DateWritable(originalDate); return originalDate.equals(dateWritable.get()); } }); assertTrue(future.get()); } finally { threadPool.shutdown(); TimeZone.setDefault(previousDefault); } } > Partitions of type 'date' behave incorrectly with daylight saving time. > ----------------------------------------------------------------------- > > Key: HIVE-8102 > URL: https://issues.apache.org/jira/browse/HIVE-8102 > Project: Hive > Issue Type: Bug > Components: Database/Schema, Serializers/Deserializers > Affects Versions: 0.13.0 > Reporter: Eli Acherkan > > On 2AM on March 28th 2014, Israel went from standard time (GMT+2) to daylight > saving time (GMT+3). > The server's timezone is Asia/Jerusalem. When creating a partition whose key > is 2014-03-28, Hive creates a partition for 2013-03-27 instead: > hive (default)> create table test (a int) partitioned by (`b_prt` date); > OK > Time taken: 0.092 seconds > hive (default)> alter table test add partition (b_prt='2014-03-28'); > OK > Time taken: 0.187 seconds > hive (default)> show partitions test; > OK > partition > b_prt=2014-03-27 > Time taken: 0.134 seconds, Fetched: 1 row(s) > It seems that the root cause is the behavior of > DateWritable.daysToMillis/dateToDays. -- This message was sent by Atlassian JIRA (v6.3.4#6332)