You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by xi...@apache.org on 2021/07/13 03:33:51 UTC

[iotdb] 01/01: Fix calendar time zone

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

xiangweiwei pushed a commit to branch calandarbug
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 4509c89fe185f16a52ae0dee64ea3118c18ecca1
Author: Alima777 <wx...@gmail.com>
AuthorDate: Tue Jul 13 05:33:14 2021 +0200

    Fix calendar time zone
---
 .../apache/iotdb/db/qp/utils/DatetimeUtils.java    |  2 ++
 .../iotdb/db/query/control/SessionManager.java     | 16 +++++++++
 .../dataset/groupby/GroupByEngineDataSet.java      |  2 ++
 .../iotdb/db/query/executor/QueryRouter.java       |  4 ++-
 .../org/apache/iotdb/db/service/TSServiceImpl.java | 39 ++++++++++------------
 .../tsfile/read/filter/GroupByMonthFilter.java     |  6 +++-
 .../tsfile/read/filter/GroupByMonthFilterTest.java | 26 ++++++++++-----
 7 files changed, 64 insertions(+), 31 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java
index 2ca9939..d360e1e 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.qp.utils;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.query.LogicalOperatorException;
+import org.apache.iotdb.db.query.control.SessionManager;
 import org.apache.iotdb.db.utils.TestOnly;
 
 import java.time.DateTimeException;
@@ -583,6 +584,7 @@ public class DatetimeUtils {
           res *= 30 * 86_400_000L;
         } else {
           Calendar calendar = Calendar.getInstance();
+          calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone());
           calendar.setTimeInMillis(currentTime);
           calendar.add(Calendar.MONTH, (int) (value));
           res = calendar.getTimeInMillis() - currentTime;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java
index 368dbfc..61b87da 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java
@@ -29,6 +29,7 @@ import java.time.ZoneId;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
+import java.util.TimeZone;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.atomic.AtomicLong;
@@ -36,6 +37,8 @@ import java.util.concurrent.atomic.AtomicLong;
 public class SessionManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(SessionManager.class);
 
+  // When the client abnormally exits, we can still know who to disconnect
+  private final ThreadLocal<Long> currSessionId = new ThreadLocal<>();
   // Record the username for every rpc connection (session).
   private final Map<Long, String> sessionIdToUsername = new ConcurrentHashMap<>();
   private final Map<Long, ZoneId> sessionIdToZoneId = new ConcurrentHashMap<>();
@@ -56,8 +59,21 @@ public class SessionManager {
     // singleton
   }
 
+  public Long getCurrSessionId() {
+    return currSessionId.get();
+  }
+
+  public void removeCurrSessionId() {
+    currSessionId.remove();
+  }
+
+  public TimeZone getCurrSessionTimeZone() {
+    return TimeZone.getTimeZone(SessionManager.getInstance().getZoneId(getCurrSessionId()));
+  }
+
   public long requestSessionId(String username, String zoneId) {
     long sessionId = sessionIdGenerator.incrementAndGet();
+    currSessionId.set(sessionId);
     sessionIdToUsername.put(sessionId, username);
     sessionIdToZoneId.put(sessionId, ZoneId.of(zoneId));
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
index 4127b1a..7194633 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.query.dataset.groupby;
 
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
 import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.SessionManager;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
@@ -159,6 +160,7 @@ public abstract class GroupByEngineDataSet extends QueryDataSet {
    */
   public long calcIntervalByMonth(long numMonths) {
     Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone());
     calendar.setTimeInMillis(startTime);
     calendar.add(Calendar.MONTH, (int) (numMonths));
     return calendar.getTimeInMillis();
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
index 6c53102..8d7c210 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.SessionManager;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByEngineDataSet;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByFillDataSet;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByTimeDataSet;
@@ -219,7 +220,8 @@ public class QueryRouter implements IQueryRouter {
               plan.getStartTime(),
               plan.getEndTime(),
               plan.isSlidingStepByMonth(),
-              plan.isIntervalByMonth())));
+              plan.isIntervalByMonth(),
+              SessionManager.getInstance().getCurrSessionTimeZone())));
     } else {
       return new GlobalTimeExpression(
           new GroupByFilter(
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 0616f61..fde203f 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -191,18 +191,14 @@ public class TSServiceImpl implements TSIService.Iface {
   protected Planner processor;
   protected IPlanExecutor executor;
 
-  private SessionManager sessionManager = SessionManager.getInstance();
-
-  // When the client abnormally exits, we can still know who to disconnect
-  private final ThreadLocal<Long> currSessionId = new ThreadLocal<>();
+  private final SessionManager sessionManager = SessionManager.getInstance();
+  private final QueryTimeManager queryTimeManager = QueryTimeManager.getInstance();
 
   public static final TSProtocolVersion CURRENT_RPC_VERSION =
       TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3;
 
   private static final AtomicInteger queryCount = new AtomicInteger(0);
 
-  private QueryTimeManager queryTimeManager = QueryTimeManager.getInstance();
-
   public TSServiceImpl() throws QueryProcessException {
     processor = new Planner();
     executor = new PlanExecutor();
@@ -261,7 +257,6 @@ public class TSServiceImpl implements TSIService.Iface {
       tsStatus = RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, "Login successfully");
 
       sessionId = sessionManager.requestSessionId(req.getUsername(), req.getZoneId());
-      currSessionId.set(sessionId);
       AUDIT_LOGGER.info("User {} opens Session-{}", req.getUsername(), sessionId);
       LOGGER.info(
           "{}: Login status: {}. User : {}",
@@ -292,7 +287,7 @@ public class TSServiceImpl implements TSIService.Iface {
     long sessionId = req.getSessionId();
     AUDIT_LOGGER.info("Session-{} is closing", sessionId);
 
-    currSessionId.remove();
+    sessionManager.removeCurrSessionId();
 
     return new TSStatus(
         !SessionTimeoutManager.getInstance().unregister(sessionId)
@@ -316,7 +311,7 @@ public class TSServiceImpl implements TSIService.Iface {
       AUDIT_LOGGER.debug(
           "{}: receive close operation from Session {}",
           IoTDBConstant.GLOBAL_DB_NAME,
-          currSessionId.get());
+          sessionManager.getCurrSessionId());
     }
 
     try {
@@ -733,7 +728,8 @@ public class TSServiceImpl implements TSIService.Iface {
           QueryFilterOptimizationException, MetadataException, IOException, InterruptedException,
           TException, AuthException {
     queryCount.incrementAndGet();
-    AUDIT_LOGGER.debug("Session {} execute Query: {}", currSessionId.get(), statement);
+    AUDIT_LOGGER.debug(
+        "Session {} execute Query: {}", sessionManager.getCurrSessionId(), statement);
     long startTime = System.currentTimeMillis();
     long queryId = -1;
     try {
@@ -1235,7 +1231,7 @@ public class TSServiceImpl implements TSIService.Iface {
   }
 
   protected void handleClientExit() {
-    Long sessionId = currSessionId.get();
+    Long sessionId = sessionManager.getCurrSessionId();
     if (sessionId != null) {
       TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
       closeSession(req);
@@ -1289,7 +1285,7 @@ public class TSServiceImpl implements TSIService.Iface {
     if (AUDIT_LOGGER.isDebugEnabled()) {
       AUDIT_LOGGER.debug(
           "Session {} insertRecords, first device {}, first time {}",
-          currSessionId.get(),
+          sessionManager.getCurrSessionId(),
           req.deviceIds.get(0),
           req.getTimestamps().get(0));
     }
@@ -1355,7 +1351,7 @@ public class TSServiceImpl implements TSIService.Iface {
     if (AUDIT_LOGGER.isDebugEnabled()) {
       AUDIT_LOGGER.debug(
           "Session {} insertRecords, device {}, first time {}",
-          currSessionId.get(),
+          sessionManager.getCurrSessionId(),
           req.deviceId,
           req.getTimestamps().get(0));
     }
@@ -1397,7 +1393,7 @@ public class TSServiceImpl implements TSIService.Iface {
     if (AUDIT_LOGGER.isDebugEnabled()) {
       AUDIT_LOGGER.debug(
           "Session {} insertRecords, first device {}, first time {}",
-          currSessionId.get(),
+          sessionManager.getCurrSessionId(),
           req.deviceIds.get(0),
           req.getTimestamps().get(0));
     }
@@ -1503,7 +1499,7 @@ public class TSServiceImpl implements TSIService.Iface {
 
       AUDIT_LOGGER.debug(
           "Session {} insertRecord, device {}, time {}",
-          currSessionId.get(),
+          sessionManager.getCurrSessionId(),
           req.getPrefixPath(),
           req.getTimestamp());
 
@@ -1532,7 +1528,7 @@ public class TSServiceImpl implements TSIService.Iface {
 
       AUDIT_LOGGER.debug(
           "Session {} insertRecord, device {}, time {}",
-          currSessionId.get(),
+          sessionManager.getCurrSessionId(),
           req.getDeviceId(),
           req.getTimestamp());
 
@@ -1709,7 +1705,8 @@ public class TSServiceImpl implements TSIService.Iface {
       }
 
       if (AUDIT_LOGGER.isDebugEnabled()) {
-        AUDIT_LOGGER.debug("Session-{} create timeseries {}", currSessionId.get(), req.getPath());
+        AUDIT_LOGGER.debug(
+            "Session-{} create timeseries {}", sessionManager.getCurrSessionId(), req.getPath());
       }
 
       CreateTimeSeriesPlan plan =
@@ -1752,7 +1749,7 @@ public class TSServiceImpl implements TSIService.Iface {
       if (AUDIT_LOGGER.isDebugEnabled()) {
         AUDIT_LOGGER.debug(
             "Session-{} create aligned timeseries {}.{}",
-            currSessionId.get(),
+            sessionManager.getCurrSessionId(),
             req.getPrefixPath(),
             req.getMeasurements());
       }
@@ -1794,7 +1791,7 @@ public class TSServiceImpl implements TSIService.Iface {
       if (AUDIT_LOGGER.isDebugEnabled()) {
         AUDIT_LOGGER.debug(
             "Session-{} create {} timeseries, the first is {}",
-            currSessionId.get(),
+            sessionManager.getCurrSessionId(),
             req.getPaths().size(),
             req.getPaths().get(0));
       }
@@ -1906,7 +1903,7 @@ public class TSServiceImpl implements TSIService.Iface {
       if (AUDIT_LOGGER.isDebugEnabled()) {
         AUDIT_LOGGER.debug(
             "Session-{} create device template {}.{}.{}.{}.{}.{}",
-            currSessionId.get(),
+            sessionManager.getCurrSessionId(),
             req.getName(),
             req.getSchemaNames(),
             req.getMeasurements(),
@@ -1964,7 +1961,7 @@ public class TSServiceImpl implements TSIService.Iface {
     if (AUDIT_LOGGER.isDebugEnabled()) {
       AUDIT_LOGGER.debug(
           "Session-{} set device template {}.{}",
-          currSessionId.get(),
+          sessionManager.getCurrSessionId(),
           req.getTemplateName(),
           req.getPrefixPath());
     }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
index df64437..213d520 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import java.util.Calendar;
 import java.util.Objects;
+import java.util.TimeZone;
 
 /**
  * GroupByMonthFilter is used to handle natural month slidingStep and interval by generating
@@ -44,9 +45,11 @@ public class GroupByMonthFilter extends GroupByFilter {
       long startTime,
       long endTime,
       boolean isSlidingStepByMonth,
-      boolean isIntervalByMonth) {
+      boolean isIntervalByMonth,
+      TimeZone timeZone) {
     super(interval, slidingStep, startTime, endTime);
     initialStartTime = startTime;
+    calendar.setTimeZone(timeZone);
     calendar.setTimeInMillis(startTime);
     this.isIntervalByMonth = isIntervalByMonth;
     this.isSlidingStepByMonth = isSlidingStepByMonth;
@@ -68,6 +71,7 @@ public class GroupByMonthFilter extends GroupByFilter {
     slidingStepsInMo = filter.slidingStepsInMo;
     initialStartTime = filter.initialStartTime;
     calendar = Calendar.getInstance();
+    calendar.setTimeZone(filter.calendar.getTimeZone());
     calendar.setTimeInMillis(filter.calendar.getTimeInMillis());
   }
 
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
index d816090..a374c74 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
@@ -24,6 +24,8 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 import org.junit.Test;
 
+import java.util.TimeZone;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -41,7 +43,8 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestSatisfy1() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_MONTH, 2 * MS_TO_MONTH, 0, END_TIME, true, true);
+        new GroupByMonthFilter(
+            MS_TO_MONTH, 2 * MS_TO_MONTH, 0, END_TIME, true, true, TimeZone.getTimeZone("+08:00"));
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -75,7 +78,8 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestSatisfy2() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true);
+        new GroupByMonthFilter(
+            MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true, TimeZone.getTimeZone("+08:00"));
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -106,7 +110,8 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestSatisfy3() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
+        new GroupByMonthFilter(
+            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, TimeZone.getTimeZone("+08:00"));
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -137,7 +142,8 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestSatisfy4() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_MONTH, MS_TO_DAY * 100, 0, END_TIME, false, true);
+        new GroupByMonthFilter(
+            MS_TO_MONTH, MS_TO_DAY * 100, 0, END_TIME, false, true, TimeZone.getTimeZone("+08:00"));
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -156,7 +162,8 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestSatisfyStartEndTime() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
+        new GroupByMonthFilter(
+            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, TimeZone.getTimeZone("+08:00"));
 
     // 1970-01-01 08:00:00 - 1970-01-02 08:00:00, timezone = GMT+08:00
     Statistics statistics = new LongStatistics();
@@ -202,7 +209,8 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestContainStartEndTime() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
+        new GroupByMonthFilter(
+            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, TimeZone.getTimeZone("+08:00"));
 
     // 1970-01-01 08:00:00 - 1970-01-02 08:00:00, timezone = GMT+08:00
     assertFalse(filter.containStartEndTime(0, MS_TO_DAY));
@@ -238,11 +246,13 @@ public class GroupByMonthFilterTest {
   @Test
   public void TestEquals() {
     GroupByMonthFilter filter =
-        new GroupByMonthFilter(MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false);
+        new GroupByMonthFilter(
+            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, TimeZone.getTimeZone("+08:00"));
     Filter filter2 = filter.copy();
     assertEquals(filter, filter2);
     GroupByMonthFilter filter3 =
-        new GroupByMonthFilter(MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true);
+        new GroupByMonthFilter(
+            MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true, TimeZone.getTimeZone("+08:00"));
     assertNotEquals(filter, filter3);
   }
 }