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

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


The following commit(s) were added to refs/heads/master by this push:
     new ff19689fa6a [IOTDB-6224] Fix group by month in non-ms precision and 
mixed unit
ff19689fa6a is described below

commit ff19689fa6a3d6502932e5924088823aa6e5f1e3
Author: Weihao Li <[email protected]>
AuthorDate: Fri Nov 3 16:11:04 2023 +0800

    [IOTDB-6224] Fix group by month in non-ms precision and mixed unit
---
 .../it/env/cluster/config/MppCommonConfig.java     |  16 ++
 .../env/cluster/config/MppSharedCommonConfig.java  |  17 ++
 .../it/env/remote/config/RemoteCommonConfig.java   |   7 +
 .../org/apache/iotdb/itbase/env/CommonConfig.java  |   4 +
 .../db/it/groupby/IoTDBGroupByNaturalMonthIT.java  | 108 +++++++++---
 .../IoTDBGroupByNaturalMonthNsPrecisionIT.java     | 107 ++++++++++++
 .../IoTDBGroupByNaturalMonthUsPrecisionIT.java     |  92 ++++++++++
 .../org/apache/iotdb/db/it/utils/TestUtils.java    |  47 +++--
 .../protocol/thrift/impl/ClientRPCServiceImpl.java |   4 +-
 .../timerangeiterator/AggrWindowIterator.java      |  91 +++++-----
 .../PreAggrWindowWithNaturalMonthIterator.java     |  17 +-
 .../TimeRangeIteratorFactory.java                  |  41 ++---
 .../execution/operator/AggregationUtil.java        |   2 -
 .../queryengine/plan/analyze/AnalyzeVisitor.java   |  16 +-
 .../plan/optimization/LimitOffsetPushDown.java     |  15 +-
 .../db/queryengine/plan/parser/ASTVisitor.java     |  53 +++---
 .../plan/parser/StatementGenerator.java            |   7 +-
 .../plan/parameter/GroupByTimeParameter.java       |  85 +++------
 .../statement/component/GroupByTimeComponent.java  |  52 ++----
 .../org/apache/iotdb/db/utils/DateTimeUtils.java   |  72 ++++++--
 .../iotdb/db/utils/TimestampPrecisionUtils.java    |  19 +-
 .../aggregation/TimeRangeIteratorTest.java         | 183 ++++++++++++++++---
 .../operator/AggregationOperatorTest.java          |   7 +-
 .../AlignedSeriesAggregationScanOperatorTest.java  |  22 ++-
 .../operator/HorizontallyConcatOperatorTest.java   |   4 +-
 .../execution/operator/OperatorMemoryTest.java     |  22 ++-
 .../operator/RawDataAggregationOperatorTest.java   |  16 +-
 .../SeriesAggregationScanOperatorTest.java         |  22 ++-
 .../SlidingWindowAggregationOperatorTest.java      |   3 +-
 .../node/process/AggregationNodeSerdeTest.java     |   3 +-
 .../node/process/GroupByLevelNodeSerdeTest.java    |   3 +-
 .../plan/node/process/GroupByTagNodeSerdeTest.java |   3 +-
 .../source/SeriesAggregationScanNodeSerdeTest.java |   3 +-
 .../DateTimeUtilsTest.java}                        |  33 +++-
 .../iotdb/tsfile/read/filter/GroupByFilter.java    |   5 +
 .../tsfile/read/filter/GroupByMonthFilter.java     | 183 +++++++++----------
 .../apache/iotdb/tsfile/utils/TimeDuration.java    | 194 +++++++++++++++++++++
 .../tsfile/read/filter/FilterSerializeTest.java    |  18 +-
 .../tsfile/read/filter/GroupByMonthFilterTest.java |  60 ++++++-
 .../iotdb/tsfile/utils/TimeDurationTest.java       |  67 +++++++
 40 files changed, 1271 insertions(+), 452 deletions(-)

diff --git 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
index 32c2539776d..121a6cf99d3 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.it.env.cluster.config;
 import org.apache.iotdb.itbase.env.CommonConfig;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 import static 
org.apache.iotdb.it.env.cluster.ClusterConstant.CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS;
 import static 
org.apache.iotdb.it.env.cluster.ClusterConstant.DATA_REGION_CONSENSUS_PROTOCOL_CLASS;
@@ -225,6 +226,21 @@ public class MppCommonConfig extends MppBaseConfig 
implements CommonConfig {
     return this;
   }
 
+  @Override
+  public TimeUnit getTimestampPrecision() {
+    String precision = properties.getProperty("timestamp_precision", "ms");
+    switch (precision) {
+      case "ms":
+        return TimeUnit.MILLISECONDS;
+      case "us":
+        return TimeUnit.MICROSECONDS;
+      case "ns":
+        return TimeUnit.NANOSECONDS;
+      default:
+        throw new UnsupportedOperationException(precision);
+    }
+  }
+
   @Override
   public CommonConfig setTimestampPrecisionCheckEnabled(boolean 
timestampPrecisionCheckEnabled) {
     setProperty(
diff --git 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
index 2d54c4dbfdb..dc89155edb4 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java
@@ -21,6 +21,8 @@ package org.apache.iotdb.it.env.cluster.config;
 
 import org.apache.iotdb.itbase.env.CommonConfig;
 
+import java.util.concurrent.TimeUnit;
+
 public class MppSharedCommonConfig implements CommonConfig {
 
   private final MppCommonConfig cnConfig;
@@ -228,6 +230,21 @@ public class MppSharedCommonConfig implements CommonConfig 
{
     return this;
   }
 
+  @Override
+  public TimeUnit getTimestampPrecision() {
+    String precision = dnConfig.properties.getProperty("timestamp_precision", 
"ms");
+    switch (precision) {
+      case "ms":
+        return TimeUnit.MILLISECONDS;
+      case "us":
+        return TimeUnit.MICROSECONDS;
+      case "ns":
+        return TimeUnit.NANOSECONDS;
+      default:
+        throw new UnsupportedOperationException(precision);
+    }
+  }
+
   @Override
   public CommonConfig setTimestampPrecisionCheckEnabled(boolean 
timestampPrecisionCheckEnabled) {
     cnConfig.setTimestampPrecisionCheckEnabled(timestampPrecisionCheckEnabled);
diff --git 
a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java
 
b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java
index 64cc2eccae5..667acca1265 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java
@@ -21,6 +21,8 @@ package org.apache.iotdb.it.env.remote.config;
 
 import org.apache.iotdb.itbase.env.CommonConfig;
 
+import java.util.concurrent.TimeUnit;
+
 public class RemoteCommonConfig implements CommonConfig {
   @Override
   public CommonConfig setMaxNumberOfPointsInPage(int maxNumberOfPointsInPage) {
@@ -163,6 +165,11 @@ public class RemoteCommonConfig implements CommonConfig {
     return this;
   }
 
+  @Override
+  public TimeUnit getTimestampPrecision() {
+    return TimeUnit.MILLISECONDS;
+  }
+
   @Override
   public CommonConfig setTimestampPrecisionCheckEnabled(boolean 
timestampPrecisionCheckEnabled) {
     return this;
diff --git 
a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java 
b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
index ef2c1678981..41f23bdae86 100644
--- 
a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
+++ 
b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
@@ -19,6 +19,8 @@
 
 package org.apache.iotdb.itbase.env;
 
+import java.util.concurrent.TimeUnit;
+
 /** This interface is used to handle properties in iotdb-common.properties. */
 public interface CommonConfig {
 
@@ -78,6 +80,8 @@ public interface CommonConfig {
 
   CommonConfig setTimestampPrecision(String timestampPrecision);
 
+  TimeUnit getTimestampPrecision();
+
   CommonConfig setTimestampPrecisionCheckEnabled(boolean 
timestampPrecisionCheckEnabled);
 
   CommonConfig setConfigNodeRatisSnapshotTriggerThreshold(int 
ratisSnapshotTriggerThreshold);
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthIT.java
index ef0e8b214a7..dc33dd9574a 100644
--- 
a/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthIT.java
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthIT.java
@@ -41,6 +41,7 @@ import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.iotdb.db.it.utils.TestUtils.assertTestFail;
 import static org.apache.iotdb.db.it.utils.TestUtils.prepareData;
@@ -54,11 +55,12 @@ import static org.junit.Assert.fail;
 @Category({LocalStandaloneIT.class, ClusterIT.class})
 public class IoTDBGroupByNaturalMonthIT {
 
-  private static final List<String> dataSet = new ArrayList<>();
+  protected static final List<String> dataSet = new ArrayList<>();
+  protected static TimeUnit currPrecision;
 
   static {
-    for (long i = 1604102400000L /*  2020-10-31 08:00:00 */;
-        i <= 1617148800000L /* 2021-03-31 08:00:00 */;
+    for (long i = 1604102400000L /*  2020-10-31 00:00:00 */;
+        i <= 1617148800000L /* 2021-03-31 00:00:00 */;
         i += 86400_000L) {
       dataSet.add("insert into root.sg1.d1(timestamp, temperature) values (" + 
i + ", 1)");
     }
@@ -75,12 +77,13 @@ public class IoTDBGroupByNaturalMonthIT {
     }
   }
 
-  private static final DateFormat df = new 
SimpleDateFormat("MM/dd/yyyy:HH:mm:ss");
+  protected static final DateFormat df = new 
SimpleDateFormat("MM/dd/yyyy:HH:mm:ss");
 
   @BeforeClass
   public static void setUp() throws Exception {
     df.setTimeZone(TimeZone.getTimeZone("GMT+00:00"));
     EnvFactory.getEnv().initClusterEnvironment();
+    currPrecision = 
EnvFactory.getEnv().getConfig().getCommonConfig().getTimestampPrecision();
     prepareData(dataSet.toArray(new String[0]));
   }
 
@@ -106,10 +109,11 @@ public class IoTDBGroupByNaturalMonthIT {
         };
     resultSetEqualTest(
         "select sum(temperature) from root.sg1.d1 "
-            + "GROUP BY ([1604102400000, 1614556800000), 1mo, 1mo)",
+            + "GROUP BY ([2020-10-31, 2021-03-01), 1mo, 1mo)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   /**
@@ -128,10 +132,11 @@ public class IoTDBGroupByNaturalMonthIT {
     };
     resultSetEqualTest(
         "select sum(temperature) from root.sg1.d1 "
-            + "GROUP BY ([1604102400000, 1614556800000), 10d, 1mo)",
+            + "GROUP BY ([2020-10-31, 2021-03-01), 10d, 1mo)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   /**
@@ -143,11 +148,11 @@ public class IoTDBGroupByNaturalMonthIT {
     String[] expectedHeader = new String[] {TIMESTAMP_STR, 
sum("root.sg1.d1.temperature")};
     String[] retArray = {"10/31/2020:00:00:00,30.0,"};
     resultSetEqualTest(
-        "select sum(temperature) from root.sg1.d1 "
-            + "GROUP BY ([1604102400000, 1606694400000), 1mo)",
+        "select sum(temperature) from root.sg1.d1 " + "GROUP BY ([2020-10-31, 
2020-11-30), 1mo)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   /**
@@ -159,10 +164,11 @@ public class IoTDBGroupByNaturalMonthIT {
     String[] expectedHeader = new String[] {TIMESTAMP_STR, 
sum("root.sg1.d1.temperature")};
     String[] retArray = {"01/31/2021:00:00:00,28.0,", 
"02/28/2021:00:00:00,31.0,"};
     resultSetEqualTest(
-        "select sum(temperature) from root.sg1.d1 GROUP BY ([1612051200000, 
1617148800000), 1mo)",
+        "select sum(temperature) from root.sg1.d1 GROUP BY ([2021-01-31, 
2021-03-31), 1mo)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   /** Test group by month with order by time desc. */
@@ -170,8 +176,12 @@ public class IoTDBGroupByNaturalMonthIT {
   public void groupByNaturalMonthFailTest() {
     assertTestFail(
         "select sum(temperature) from root.sg1.d1 "
-            + "GROUP BY ([1612051200000, 1617148800000), 1mo) order by time 
desc",
+            + "GROUP BY ([2021-01-31, 2021-03-31), 1mo) order by time desc",
         "doesn't support order by time desc now.");
+
+    assertTestFail(
+        "select sum(temperature) from root.sg1.d1 GROUP BY ([1970-01-01, 
2970-01-01), 40d, 1mo)",
+        "The time windows may exceed 10000, please ensure your input.");
   }
 
   /** StartTime: now() - 1mo, EndTime: now(). */
@@ -217,10 +227,11 @@ public class IoTDBGroupByNaturalMonthIT {
     };
     resultSetEqualTest(
         "select sum(temperature) from root.sg1.d1 "
-            + "GROUP BY ([1604102400000, 1614556800000), 2mo, 1mo)",
+            + "GROUP BY ([2020-10-31, 2021-03-01), 2mo, 1mo)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   @Test
@@ -243,10 +254,11 @@ public class IoTDBGroupByNaturalMonthIT {
     };
     resultSetEqualTest(
         "select sum(temperature) from root.sg1.d1 "
-            + "GROUP BY ([1604102400000, 1614556800000), 1mo, 10d)",
+            + "GROUP BY ([2020-10-31, 2021-03-01), 1mo, 10d)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   @Test
@@ -263,7 +275,8 @@ public class IoTDBGroupByNaturalMonthIT {
         "select count(s1) from root.test.d1 " + "group by ([2023-01-01, 
2027-01-01), 1y)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   @Test
@@ -280,7 +293,8 @@ public class IoTDBGroupByNaturalMonthIT {
         "select count(s1) from root.test.d1 " + "group by ((2023-01-01, 
2027-01-01], 1y)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   @Test
@@ -301,7 +315,8 @@ public class IoTDBGroupByNaturalMonthIT {
         "select count(s1) from root.test.d1 " + "group by ([2023-01-01, 
2027-01-01), 1y, 6mo)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   @Test
@@ -318,7 +333,8 @@ public class IoTDBGroupByNaturalMonthIT {
         "select count(s1) from root.test.d1 " + "group by ([2023-01-01, 
2027-01-01), 2y, 1y)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
   }
 
   @Test
@@ -335,6 +351,52 @@ public class IoTDBGroupByNaturalMonthIT {
         "select count(s1) from root.test.d1 " + "group by ([2023-01-01, 
2027-01-01), 6mo, 1y)",
         expectedHeader,
         retArray,
-        df);
+        df,
+        currPrecision);
+  }
+
+  @Test
+  public void groupByNaturalMonthWithMixedUnit1() {
+    String[] expectedHeader = new String[] {TIMESTAMP_STR, 
count("root.test.d1.s1")};
+    String[] retArray =
+        new String[] {
+          // [01-28, 03-01)
+          "01/28/2023:00:00:00,1,",
+          // [03-01, 04-02)
+          "03/01/2023:00:00:00,2,",
+          // [04-02, 05-03)
+          "04/02/2023:00:00:00,1,",
+          // [05-03, 05-29)
+          "05/03/2023:00:00:00,0,"
+        };
+    resultSetEqualTest(
+        "select count(s1) from root.test.d1 " + "group by ([2023-01-28, 
2023-05-29), 1mo1d)",
+        expectedHeader,
+        retArray,
+        df,
+        currPrecision);
+  }
+
+  @Test
+  public void groupByNaturalMonthWithMixedUnit2() {
+    String[] expectedHeader = new String[] {TIMESTAMP_STR, 
count("root.test.d1.s1")};
+    String[] retArray =
+        new String[] {
+          // [01-28, 03-01)
+          "1674864000000,1,",
+          // [03-01, 04-02)
+          "1677628800000,2,",
+          // [04-02, 05-03)
+          "1680393600000,1,",
+          // [05-03, 05-29)
+          "1683072000000,0,"
+        };
+    // the part in timeDuration finer than current time precision will be 
discarded
+    resultSetEqualTest(
+        "select count(s1) from root.test.d1 " + "group by ([2023-01-28, 
2023-05-29), 1mo1d1ns)",
+        expectedHeader,
+        retArray,
+        null,
+        currPrecision);
   }
 }
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthNsPrecisionIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthNsPrecisionIT.java
new file mode 100644
index 00000000000..8ff16207304
--- /dev/null
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthNsPrecisionIT.java
@@ -0,0 +1,107 @@
+/*
+ * 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.iotdb.db.it.groupby;
+
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+import org.apache.iotdb.itbase.category.LocalStandaloneIT;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import static org.apache.iotdb.db.it.utils.TestUtils.assertTestFail;
+import static org.apache.iotdb.db.it.utils.TestUtils.prepareData;
+import static org.apache.iotdb.db.it.utils.TestUtils.resultSetEqualTest;
+import static org.apache.iotdb.itbase.constant.TestConstant.TIMESTAMP_STR;
+import static org.apache.iotdb.itbase.constant.TestConstant.count;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({LocalStandaloneIT.class, ClusterIT.class})
+public class IoTDBGroupByNaturalMonthNsPrecisionIT extends 
IoTDBGroupByNaturalMonthIT {
+  static {
+    for (long i = 1604102400000L /*  2020-10-31 00:00:00 */;
+        i <= 1617148800000L /* 2021-03-31 00:00:00 */;
+        i += 86400_000L) {
+      dataSet.add(
+          "insert into root.sg1.d1(timestamp, temperature) values (" + i * 
1000000 + ", 1)");
+    }
+
+    // TimeRange: [2023-01-01 00:00:00, 2027-01-01 00:00:00]
+    // insert a record each first day of month
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(TimeZone.getTimeZone("+00:00"));
+    calendar.setTimeInMillis(1672531200000L);
+    for (long i = calendar.getTimeInMillis();
+        i <= 1798761600000L;
+        calendar.add(Calendar.MONTH, 1), i = calendar.getTimeInMillis()) {
+      dataSet.add("insert into root.test.d1(timestamp, s1) values (" + i * 
1000000 + ", 1)");
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    df.setTimeZone(TimeZone.getTimeZone("GMT+00:00"));
+    
EnvFactory.getEnv().getConfig().getCommonConfig().setTimestampPrecision("ns");
+    EnvFactory.getEnv().initClusterEnvironment();
+    currPrecision = 
EnvFactory.getEnv().getConfig().getCommonConfig().getTimestampPrecision();
+    prepareData(dataSet.toArray(new String[0]));
+  }
+
+  @Test
+  public void groupByNaturalMonthFailTest() {
+    assertTestFail(
+        "select sum(temperature) from root.sg1.d1 "
+            + "GROUP BY ([2021-01-31, 2021-03-31), 1mo) order by time desc",
+        "doesn't support order by time desc now.");
+
+    // 2970-01-01 in ns precision has exceeded field of long number
+    assertTestFail(
+        "select sum(temperature) from root.sg1.d1 GROUP BY ([1970-01-01, 
2970-01-01), 40d, 1mo)",
+        "Input time format 2970-01-01 error.");
+  }
+
+  @Test
+  public void groupByNaturalMonthWithMixedUnit2() {
+    String[] expectedHeader = new String[] {TIMESTAMP_STR, 
count("root.test.d1.s1")};
+    String[] retArray =
+        new String[] {
+          // [01-28, 03-01 + 1ns)
+          "1674864000000000000,2,",
+          // [03-01 + 1ns, 04-02 + 2ns)
+          "1677628800000000001,1,",
+          // [04-02 + 2ns, 05-03 + 3ns)
+          "1680393600000000002,1,",
+          // [05-03 + 3ns, 05-29 + 4ns)
+          "1683072000000000003,0,"
+        };
+    // the part in timeDuration finer than current time precision will be 
discarded
+    resultSetEqualTest(
+        "select count(s1) from root.test.d1 " + "group by ([2023-01-28, 
2023-05-29), 1mo1d1ns)",
+        expectedHeader,
+        retArray,
+        null,
+        currPrecision);
+  }
+}
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthUsPrecisionIT.java
 
b/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthUsPrecisionIT.java
new file mode 100644
index 00000000000..0f55af25da8
--- /dev/null
+++ 
b/integration-test/src/test/java/org/apache/iotdb/db/it/groupby/IoTDBGroupByNaturalMonthUsPrecisionIT.java
@@ -0,0 +1,92 @@
+/*
+ * 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.iotdb.db.it.groupby;
+
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+import org.apache.iotdb.itbase.category.LocalStandaloneIT;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import static org.apache.iotdb.db.it.utils.TestUtils.prepareData;
+import static org.apache.iotdb.db.it.utils.TestUtils.resultSetEqualTest;
+import static org.apache.iotdb.itbase.constant.TestConstant.TIMESTAMP_STR;
+import static org.apache.iotdb.itbase.constant.TestConstant.count;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({LocalStandaloneIT.class, ClusterIT.class})
+public class IoTDBGroupByNaturalMonthUsPrecisionIT extends 
IoTDBGroupByNaturalMonthIT {
+  static {
+    for (long i = 1604102400000L /*  2020-10-31 00:00:00 */;
+        i <= 1617148800000L /* 2021-03-31 00:00:00 */;
+        i += 86400_000L) {
+      dataSet.add("insert into root.sg1.d1(timestamp, temperature) values (" + 
i * 1000 + ", 1)");
+    }
+
+    // TimeRange: [2023-01-01 00:00:00, 2027-01-01 00:00:00]
+    // insert a record each first day of month
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(TimeZone.getTimeZone("+00:00"));
+    calendar.setTimeInMillis(1672531200000L);
+    for (long i = calendar.getTimeInMillis();
+        i <= 1798761600000L;
+        calendar.add(Calendar.MONTH, 1), i = calendar.getTimeInMillis()) {
+      dataSet.add("insert into root.test.d1(timestamp, s1) values (" + i * 
1000 + ", 1)");
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    df.setTimeZone(TimeZone.getTimeZone("GMT+00:00"));
+    
EnvFactory.getEnv().getConfig().getCommonConfig().setTimestampPrecision("us");
+    EnvFactory.getEnv().initClusterEnvironment();
+    currPrecision = 
EnvFactory.getEnv().getConfig().getCommonConfig().getTimestampPrecision();
+    prepareData(dataSet.toArray(new String[0]));
+  }
+
+  @Test
+  public void groupByNaturalMonthWithMixedUnit2() {
+    String[] expectedHeader = new String[] {TIMESTAMP_STR, 
count("root.test.d1.s1")};
+    String[] retArray =
+        new String[] {
+          // [01-28, 03-01)
+          "1674864000000000,1,",
+          // [03-01, 04-02)
+          "1677628800000000,2,",
+          // [04-02, 05-03)
+          "1680393600000000,1,",
+          // [05-03, 05-29)
+          "1683072000000000,0,"
+        };
+    // the part in timeDuration finer than current time precision will be 
discarded
+    resultSetEqualTest(
+        "select count(s1) from root.test.d1 " + "group by ([2023-01-28, 
2023-05-29), 1mo1d1ns)",
+        expectedHeader,
+        retArray,
+        null,
+        currPrecision);
+  }
+}
diff --git 
a/integration-test/src/test/java/org/apache/iotdb/db/it/utils/TestUtils.java 
b/integration-test/src/test/java/org/apache/iotdb/db/it/utils/TestUtils.java
index 4b9be28d4a1..4e48ee71ffb 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/utils/TestUtils.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/utils/TestUtils.java
@@ -134,14 +134,13 @@ public class TestUtils {
       String[] expectedRetArray,
       DateFormat df,
       String userName,
-      String password) {
+      String password,
+      TimeUnit currPrecision) {
     try (Connection connection = EnvFactory.getEnv().getConnection(userName, 
password);
         Statement statement = connection.createStatement()) {
-      if (df != null) {
-        connection.setClientInfo("time_zone", "+00:00");
-      }
+      connection.setClientInfo("time_zone", "+00:00");
       try (ResultSet resultSet = statement.executeQuery(sql)) {
-        assertResultSetEqual(resultSet, expectedHeader, expectedRetArray, df);
+        assertResultSetEqual(resultSet, expectedHeader, expectedRetArray, df, 
currPrecision);
       }
     } catch (SQLException e) {
       e.printStackTrace();
@@ -151,7 +150,8 @@ public class TestUtils {
 
   public static void resultSetEqualTest(
       String sql, String expectedHeader, String[] expectedRetArray) {
-    resultSetEqualTest(sql, expectedHeader, expectedRetArray, null, "root", 
"root");
+    resultSetEqualTest(
+        sql, expectedHeader, expectedRetArray, null, "root", "root", 
TimeUnit.MILLISECONDS);
   }
 
   public static void resultSetEqualTest(
@@ -174,7 +174,21 @@ public class TestUtils {
     for (String s : expectedHeader) {
       header.append(s).append(",");
     }
-    resultSetEqualTest(sql, header.toString(), expectedRetArray, df, "root", 
"root");
+    resultSetEqualTest(
+        sql, header.toString(), expectedRetArray, df, "root", "root", 
TimeUnit.MILLISECONDS);
+  }
+
+  public static void resultSetEqualTest(
+      String sql,
+      String[] expectedHeader,
+      String[] expectedRetArray,
+      DateFormat df,
+      TimeUnit currPrecision) {
+    StringBuilder header = new StringBuilder();
+    for (String s : expectedHeader) {
+      header.append(s).append(",");
+    }
+    resultSetEqualTest(sql, header.toString(), expectedRetArray, df, "root", 
"root", currPrecision);
   }
 
   public static void resultSetEqualTest(
@@ -188,7 +202,8 @@ public class TestUtils {
     for (String s : expectedHeader) {
       header.append(s).append(",");
     }
-    resultSetEqualTest(sql, header.toString(), expectedRetArray, df, userName, 
password);
+    resultSetEqualTest(
+        sql, header.toString(), expectedRetArray, df, userName, password, 
TimeUnit.MILLISECONDS);
   }
 
   public static void resultSetEqualWithDescOrderTest(
@@ -269,7 +284,8 @@ public class TestUtils {
 
   public static void assertResultSetEqual(
       ResultSet actualResultSet, String expectedHeader, String[] 
expectedRetArray) {
-    assertResultSetEqual(actualResultSet, expectedHeader, expectedRetArray, 
null);
+    assertResultSetEqual(
+        actualResultSet, expectedHeader, expectedRetArray, null, 
TimeUnit.MILLISECONDS);
   }
 
   public static void assertResultSetEqual(
@@ -299,7 +315,11 @@ public class TestUtils {
   }
 
   public static void assertResultSetEqual(
-      ResultSet actualResultSet, String expectedHeader, String[] 
expectedRetArray, DateFormat df) {
+      ResultSet actualResultSet,
+      String expectedHeader,
+      String[] expectedRetArray,
+      DateFormat df,
+      TimeUnit currPrecision) {
     try {
       ResultSetMetaData resultSetMetaData = actualResultSet.getMetaData();
       StringBuilder header = new StringBuilder();
@@ -312,7 +332,12 @@ public class TestUtils {
       while (actualResultSet.next()) {
         StringBuilder builder = new StringBuilder();
         if (df != null) {
-          
builder.append(df.format(Long.parseLong(actualResultSet.getString(1)))).append(",");
+          builder
+              .append(
+                  df.format(
+                      TimeUnit.MILLISECONDS.convert(
+                          Long.parseLong(actualResultSet.getString(1)), 
currPrecision)))
+              .append(",");
         } else {
           builder.append(actualResultSet.getString(1)).append(",");
         }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java
index f748ced7528..98a20ba5074 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java
@@ -172,6 +172,7 @@ import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
@@ -673,7 +674,8 @@ public class ClientRPCServiceImpl implements 
IClientRPCServiceWithHandler {
             Collections.singletonList(new InputLocation[] {new 
InputLocation(0, 0)}));
 
     GroupByTimeParameter groupByTimeParameter =
-        new GroupByTimeParameter(startTime, endTme, interval, interval, true);
+        new GroupByTimeParameter(
+            startTime, endTme, new TimeDuration(0, interval), new 
TimeDuration(0, interval), true);
 
     IMeasurementSchema measurementSchema = new MeasurementSchema(measurement, 
dataType);
     AbstractSeriesAggregationScanOperator operator;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/AggrWindowIterator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/AggrWindowIterator.java
index 0ba0153d525..25264d58e4f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/AggrWindowIterator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/AggrWindowIterator.java
@@ -20,9 +20,9 @@
 package 
org.apache.iotdb.db.queryengine.execution.aggregation.timerangeiterator;
 
 import org.apache.iotdb.db.utils.DateTimeUtils;
+import org.apache.iotdb.db.utils.TimestampPrecisionUtils;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
-
-import static org.apache.iotdb.db.utils.DateTimeUtils.MS_TO_MONTH;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 /**
  * This class iteratively generates aggregated time windows.
@@ -34,12 +34,10 @@ public class AggrWindowIterator implements 
ITimeRangeIterator {
 
   private final long startTime;
   private final long endTime;
-  private final long interval;
-  private final long slidingStep;
+  private final TimeDuration interval;
+  private final TimeDuration slidingStep;
 
   private final boolean isAscending;
-  private final boolean isSlidingStepByMonth;
-  private final boolean isIntervalByMonth;
   private final boolean leftCRightO;
 
   private TimeRange curTimeRange;
@@ -49,19 +47,15 @@ public class AggrWindowIterator implements 
ITimeRangeIterator {
   public AggrWindowIterator(
       long startTime,
       long endTime,
-      long interval,
-      long slidingStep,
+      TimeDuration interval,
+      TimeDuration slidingStep,
       boolean isAscending,
-      boolean isSlidingStepByMonth,
-      boolean isIntervalByMonth,
       boolean leftCRightO) {
     this.startTime = startTime;
     this.endTime = endTime;
     this.interval = interval;
     this.slidingStep = slidingStep;
     this.isAscending = isAscending;
-    this.isSlidingStepByMonth = isSlidingStepByMonth;
-    this.isIntervalByMonth = isIntervalByMonth;
     this.leftCRightO = leftCRightO;
   }
 
@@ -76,12 +70,13 @@ public class AggrWindowIterator implements 
ITimeRangeIterator {
 
   private TimeRange getLeftmostTimeRange() {
     long retEndTime;
-    if (isIntervalByMonth) {
+    if (interval.containsMonth()) {
       // calculate interval length by natural month based on startTime
       // ie. startTIme = 1/31, interval = 1mo, curEndTime will be set to 2/29
-      retEndTime = Math.min(DateTimeUtils.calcIntervalByMonth(startTime, 
interval), endTime);
+      retEndTime =
+          Math.min(DateTimeUtils.calcPositiveIntervalByMonth(startTime, 
interval, 1), endTime);
     } else {
-      retEndTime = Math.min(startTime + interval, endTime);
+      retEndTime = Math.min(startTime + interval.nonMonthDuration, endTime);
     }
     return new TimeRange(startTime, retEndTime);
   }
@@ -92,24 +87,31 @@ public class AggrWindowIterator implements 
ITimeRangeIterator {
     long queryRange = endTime - startTime;
     long intervalNum;
 
-    if (isSlidingStepByMonth) {
-      intervalNum = (long) Math.ceil(queryRange / (double) (slidingStep * 
MS_TO_MONTH));
-      retStartTime = DateTimeUtils.calcIntervalByMonth(startTime, intervalNum 
* slidingStep);
-      while (retStartTime >= endTime) {
-        intervalNum -= 1;
-        retStartTime = DateTimeUtils.calcIntervalByMonth(startTime, 
intervalNum * slidingStep);
+    if (slidingStep.containsMonth()) {
+      intervalNum =
+          (long)
+              Math.ceil(
+                  (double) queryRange
+                      / 
(slidingStep.getMaxTotalDuration(TimestampPrecisionUtils.currPrecision)));
+      long tempRetStartTime =
+          DateTimeUtils.calcPositiveIntervalByMonth(startTime, slidingStep, 
intervalNum - 1);
+      retStartTime = tempRetStartTime;
+      while (tempRetStartTime < endTime) {
+        retStartTime = tempRetStartTime;
+        tempRetStartTime = 
DateTimeUtils.calcPositiveIntervalByMonth(retStartTime, slidingStep, 1);
       }
     } else {
-      intervalNum = (long) Math.ceil(queryRange / (double) slidingStep);
-      retStartTime = slidingStep * (intervalNum - 1) + startTime;
+      intervalNum = (long) Math.ceil(queryRange / (double) 
slidingStep.nonMonthDuration);
+      retStartTime = slidingStep.nonMonthDuration * (intervalNum - 1) + 
startTime;
     }
 
-    if (isIntervalByMonth) {
+    if (interval.containsMonth()) {
       // calculate interval length by natural month based on curStartTime
       // ie. startTIme = 1/31, interval = 1mo, curEndTime will be set to 2/29
-      retEndTime = Math.min(DateTimeUtils.calcIntervalByMonth(retStartTime, 
interval), endTime);
+      retEndTime =
+          Math.min(DateTimeUtils.calcPositiveIntervalByMonth(retStartTime, 
interval, 1), endTime);
     } else {
-      retEndTime = Math.min(retStartTime + interval, endTime);
+      retEndTime = Math.min(retStartTime + interval.nonMonthDuration, endTime);
     }
     return new TimeRange(retStartTime, retEndTime);
   }
@@ -129,30 +131,30 @@ public class AggrWindowIterator implements 
ITimeRangeIterator {
     long retEndTime;
     long curStartTime = curTimeRange.getMin();
     if (isAscending) {
-      if (isSlidingStepByMonth) {
-        retStartTime = DateTimeUtils.calcIntervalByMonth(curStartTime, (int) 
(slidingStep));
+      if (slidingStep.containsMonth()) {
+        retStartTime = DateTimeUtils.calcPositiveIntervalByMonth(curStartTime, 
slidingStep, 1);
       } else {
-        retStartTime = curStartTime + slidingStep;
+        retStartTime = curStartTime + slidingStep.nonMonthDuration;
       }
       // This is an open interval , [0-100)
       if (retStartTime >= endTime) {
         return false;
       }
     } else {
-      if (isSlidingStepByMonth) {
-        retStartTime = DateTimeUtils.calcIntervalByMonth(curStartTime, (int) 
(-slidingStep));
+      if (slidingStep.containsMonth()) {
+        retStartTime = DateTimeUtils.calcNegativeIntervalByMonth(curStartTime, 
slidingStep);
       } else {
-        retStartTime = curStartTime - slidingStep;
+        retStartTime = curStartTime - slidingStep.nonMonthDuration;
       }
       if (retStartTime < startTime) {
         return false;
       }
     }
 
-    if (isIntervalByMonth) {
-      retEndTime = DateTimeUtils.calcIntervalByMonth(retStartTime, (int) 
(interval));
+    if (interval.containsMonth()) {
+      retEndTime = DateTimeUtils.calcPositiveIntervalByMonth(retStartTime, 
interval, 1);
     } else {
-      retEndTime = retStartTime + interval;
+      retEndTime = retStartTime + interval.nonMonthDuration;
     }
     retEndTime = Math.min(retEndTime, endTime);
     curTimeRange = new TimeRange(retStartTime, retEndTime);
@@ -184,15 +186,20 @@ public class AggrWindowIterator implements 
ITimeRangeIterator {
     long queryRange = endTime - startTime;
     long intervalNum;
 
-    if (isSlidingStepByMonth) {
-      intervalNum = (long) Math.ceil(queryRange / (double) (slidingStep * 
MS_TO_MONTH));
-      long retStartTime = DateTimeUtils.calcIntervalByMonth(startTime, 
intervalNum * slidingStep);
-      while (retStartTime > endTime) {
-        intervalNum -= 1;
-        retStartTime = DateTimeUtils.calcIntervalByMonth(startTime, 
intervalNum * slidingStep);
+    if (slidingStep.containsMonth()) {
+      intervalNum =
+          (long)
+              Math.ceil(
+                  (double) queryRange
+                      / 
(slidingStep.getMaxTotalDuration(TimestampPrecisionUtils.currPrecision)));
+      long retStartTime =
+          DateTimeUtils.calcPositiveIntervalByMonth(startTime, slidingStep, 
intervalNum);
+      while (retStartTime < endTime) {
+        intervalNum++;
+        retStartTime = DateTimeUtils.calcPositiveIntervalByMonth(retStartTime, 
slidingStep, 1);
       }
     } else {
-      intervalNum = (long) Math.ceil(queryRange / (double) slidingStep);
+      intervalNum = (long) Math.ceil(queryRange / (double) 
slidingStep.nonMonthDuration);
     }
     return intervalNum;
   }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/PreAggrWindowWithNaturalMonthIterator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/PreAggrWindowWithNaturalMonthIterator.java
index 194e9a58d18..74c7eed0f7b 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/PreAggrWindowWithNaturalMonthIterator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/PreAggrWindowWithNaturalMonthIterator.java
@@ -21,6 +21,7 @@ package 
org.apache.iotdb.db.queryengine.execution.aggregation.timerangeiterator;
 
 import org.apache.iotdb.db.utils.datastructure.TimeSelector;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 public class PreAggrWindowWithNaturalMonthIterator implements 
ITimeRangeIterator {
 
@@ -41,24 +42,14 @@ public class PreAggrWindowWithNaturalMonthIterator 
implements ITimeRangeIterator
   public PreAggrWindowWithNaturalMonthIterator(
       long startTime,
       long endTime,
-      long interval,
-      long slidingStep,
+      TimeDuration interval,
+      TimeDuration slidingStep,
       boolean isAscending,
-      boolean isSlidingStepByMonth,
-      boolean isIntervalByMonth,
       boolean leftCRightO) {
     this.isAscending = isAscending;
     this.timeBoundaryHeap = new TimeSelector(HEAP_MAX_SIZE, isAscending);
     this.aggrWindowIterator =
-        new AggrWindowIterator(
-            startTime,
-            endTime,
-            interval,
-            slidingStep,
-            isAscending,
-            isSlidingStepByMonth,
-            isIntervalByMonth,
-            leftCRightO);
+        new AggrWindowIterator(startTime, endTime, interval, slidingStep, 
isAscending, leftCRightO);
     this.leftCRightO = leftCRightO;
     initHeap();
   }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/TimeRangeIteratorFactory.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/TimeRangeIteratorFactory.java
index 22b51b7ba77..18cd715f06f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/TimeRangeIteratorFactory.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/aggregation/timerangeiterator/TimeRangeIteratorFactory.java
@@ -19,7 +19,8 @@
 
 package 
org.apache.iotdb.db.queryengine.execution.aggregation.timerangeiterator;
 
-import static org.apache.iotdb.db.utils.DateTimeUtils.MS_TO_MONTH;
+import org.apache.iotdb.db.utils.TimestampPrecisionUtils;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 public class TimeRangeIteratorFactory {
 
@@ -34,43 +35,29 @@ public class TimeRangeIteratorFactory {
   public static ITimeRangeIterator getTimeRangeIterator(
       long startTime,
       long endTime,
-      long interval,
-      long slidingStep,
+      TimeDuration interval,
+      TimeDuration slidingStep,
       boolean isAscending,
-      boolean isIntervalByMonth,
-      boolean isSlidingStepByMonth,
       boolean leftCRightO,
       boolean outputPartialTimeWindow) {
-    long originInterval = interval;
-    long originSlidingStep = slidingStep;
-    interval = isIntervalByMonth ? interval / MS_TO_MONTH : interval;
-    slidingStep = isSlidingStepByMonth ? slidingStep / MS_TO_MONTH : 
slidingStep;
-
-    if (outputPartialTimeWindow && originInterval > originSlidingStep) {
-      if (!isIntervalByMonth && !isSlidingStepByMonth) {
+    if (outputPartialTimeWindow
+        && interval.getTotalDuration(TimestampPrecisionUtils.currPrecision)
+            > 
slidingStep.getTotalDuration(TimestampPrecisionUtils.currPrecision)) {
+      if (!interval.containsMonth() && !slidingStep.containsMonth()) {
         return new PreAggrWindowIterator(
-            startTime, endTime, interval, slidingStep, isAscending, 
leftCRightO);
-      } else {
-        return new PreAggrWindowWithNaturalMonthIterator(
             startTime,
             endTime,
-            interval,
-            slidingStep,
+            interval.nonMonthDuration,
+            slidingStep.nonMonthDuration,
             isAscending,
-            isSlidingStepByMonth,
-            isIntervalByMonth,
             leftCRightO);
+      } else {
+        return new PreAggrWindowWithNaturalMonthIterator(
+            startTime, endTime, interval, slidingStep, isAscending, 
leftCRightO);
       }
     } else {
       return new AggrWindowIterator(
-          startTime,
-          endTime,
-          interval,
-          slidingStep,
-          isAscending,
-          isSlidingStepByMonth,
-          isIntervalByMonth,
-          leftCRightO);
+          startTime, endTime, interval, slidingStep, isAscending, leftCRightO);
     }
   }
 }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java
index 6b0ea5f7941..b72ad3c6da6 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationUtil.java
@@ -76,8 +76,6 @@ public class AggregationUtil {
           groupByTimeParameter.getInterval(),
           groupByTimeParameter.getSlidingStep(),
           ascending,
-          groupByTimeParameter.isIntervalByMonth(),
-          groupByTimeParameter.isSlidingStepByMonth(),
           groupByTimeParameter.isLeftCRightO(),
           outputPartialTimeWindow);
     }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
index 6735169db76..96e8c9f5a42 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java
@@ -137,6 +137,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainStatement;
 import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement;
 import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement;
 import org.apache.iotdb.db.schemaengine.template.Template;
+import org.apache.iotdb.db.utils.TimestampPrecisionUtils;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.enums.TSDataType;
@@ -1752,7 +1753,8 @@ public class AnalyzeVisitor extends 
StatementVisitor<Analysis, MPPQueryContext>
     }
 
     GroupByTimeComponent groupByTimeComponent = 
queryStatement.getGroupByTimeComponent();
-    if ((groupByTimeComponent.isIntervalByMonth() || 
groupByTimeComponent.isSlidingStepByMonth())
+    if ((groupByTimeComponent.getInterval().containsMonth()
+            || groupByTimeComponent.getSlidingStep().containsMonth())
         && queryStatement.getResultTimeOrder() == Ordering.DESC) {
       throw new SemanticException("Group by month doesn't support order by 
time desc now.");
     }
@@ -2985,19 +2987,19 @@ public class AnalyzeVisitor extends 
StatementVisitor<Analysis, MPPQueryContext>
         groupByTimeComponent.isLeftCRightO()
             ? groupByTimeComponent.getEndTime()
             : groupByTimeComponent.getEndTime() + 1;
-    if (groupByTimeComponent.isIntervalByMonth() || 
groupByTimeComponent.isSlidingStepByMonth()) {
+    if (groupByTimeComponent.getInterval().containsMonth()
+        || groupByTimeComponent.getSlidingStep().containsMonth()) {
       return new GroupByMonthFilter(
           groupByTimeComponent.getInterval(),
           groupByTimeComponent.getSlidingStep(),
           startTime,
           endTime,
-          groupByTimeComponent.isSlidingStepByMonth(),
-          groupByTimeComponent.isIntervalByMonth(),
-          TimeZone.getTimeZone("+00:00"));
+          TimeZone.getTimeZone("+00:00"),
+          TimestampPrecisionUtils.currPrecision);
     } else {
       return new GroupByFilter(
-          groupByTimeComponent.getInterval(),
-          groupByTimeComponent.getSlidingStep(),
+          groupByTimeComponent.getInterval().nonMonthDuration,
+          groupByTimeComponent.getSlidingStep().nonMonthDuration,
           startTime,
           endTime);
     }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java
index 37dd6ff17a5..c4d6815c904 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java
@@ -43,6 +43,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.component.Ordering;
 import org.apache.iotdb.db.queryengine.plan.statement.crud.QueryStatement;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -264,8 +265,8 @@ public class LimitOffsetPushDown implements PlanOptimizer {
     GroupByTimeComponent groupByTimeComponent = 
queryStatement.getGroupByTimeComponent();
     long startTime = groupByTimeComponent.getStartTime();
     long endTime = groupByTimeComponent.getEndTime();
-    long step = groupByTimeComponent.getSlidingStep();
-    long interval = groupByTimeComponent.getInterval();
+    long step = groupByTimeComponent.getSlidingStep().nonMonthDuration;
+    long interval = groupByTimeComponent.getInterval().nonMonthDuration;
 
     long size = (endTime - startTime + step - 1) / step;
     if (size > queryStatement.getRowOffset()) {
@@ -309,12 +310,14 @@ public class LimitOffsetPushDown implements PlanOptimizer 
{
   public static List<PartialPath> pushDownLimitOffsetInGroupByTimeForDevice(
       List<PartialPath> deviceNames, QueryStatement queryStatement) {
     GroupByTimeComponent groupByTimeComponent = 
queryStatement.getGroupByTimeComponent();
+    if (groupByTimeComponent.getInterval().containsMonth()
+        || groupByTimeComponent.getSlidingStep().containsMonth()) {
+      return Collections.emptyList();
+    }
     long startTime = groupByTimeComponent.getStartTime();
     long endTime = groupByTimeComponent.getEndTime();
-
-    long size =
-        (endTime - startTime + groupByTimeComponent.getSlidingStep() - 1)
-            / groupByTimeComponent.getSlidingStep();
+    long slidingStep = groupByTimeComponent.getSlidingStep().nonMonthDuration;
+    long size = (endTime - startTime + slidingStep - 1) / slidingStep;
     if (size == 0 || size * deviceNames.size() <= 
queryStatement.getRowOffset()) {
       // resultSet is empty
       queryStatement.setResultSetEmpty(true);
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
index 0a1cc222bdb..108534f4bb7 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java
@@ -202,6 +202,7 @@ import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import com.google.common.collect.ImmutableSet;
 import org.antlr.v4.runtime.tree.TerminalNode;
@@ -225,6 +226,7 @@ import java.util.stream.Collectors;
 import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_RESULT_NODES;
 import static 
org.apache.iotdb.db.queryengine.plan.optimization.LimitOffsetPushDown.canPushDownLimitOffsetToGroupByTime;
 import static 
org.apache.iotdb.db.queryengine.plan.optimization.LimitOffsetPushDown.pushDownLimitOffsetToTimeParameter;
+import static org.apache.iotdb.db.utils.TimestampPrecisionUtils.currPrecision;
 import static org.apache.iotdb.db.utils.constant.SqlConstant.CAST_FUNCTION;
 import static org.apache.iotdb.db.utils.constant.SqlConstant.CAST_TYPE;
 import static org.apache.iotdb.db.utils.constant.SqlConstant.REPLACE_FROM;
@@ -940,7 +942,8 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
             "CQ: At least one of the parameters `every_interval` and 
`group_by_interval` needs to be specified.");
       }
 
-      long interval = queryStatement.getGroupByTimeComponent().getInterval();
+      long interval =
+          
queryStatement.getGroupByTimeComponent().getInterval().getTotalDuration(currPrecision);
       statement.setEveryInterval(interval);
       statement.setStartTimeOffset(interval);
     }
@@ -963,7 +966,8 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
         throw new SemanticException(
             "CQ: At least one of the parameters `every_interval` and 
`group_by_interval` needs to be specified.");
       }
-      
statement.setEveryInterval(queryStatement.getGroupByTimeComponent().getInterval());
+      statement.setEveryInterval(
+          
queryStatement.getGroupByTimeComponent().getInterval().getTotalDuration(currPrecision));
     }
 
     if (ctx.BOUNDARY() != null) {
@@ -1468,22 +1472,35 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
     }
 
     // Parse time interval
-    groupByTimeComponent.setInterval(
-        parseTimeIntervalOrSlidingStep(ctx.interval.getText(), true, 
groupByTimeComponent));
-    if (groupByTimeComponent.getInterval() <= 0) {
+    
groupByTimeComponent.setInterval(DateTimeUtils.constructTimeDuration(ctx.interval.getText()));
+    groupByTimeComponent.setOriginalInterval(ctx.interval.getText());
+    if (groupByTimeComponent.getInterval().monthDuration == 0
+        && groupByTimeComponent.getInterval().nonMonthDuration == 0) {
       throw new SemanticException(
           "The second parameter time interval should be a positive integer.");
     }
 
     // parse sliding step
     if (ctx.step != null) {
-      groupByTimeComponent.setSlidingStep(
-          parseTimeIntervalOrSlidingStep(ctx.step.getText(), false, 
groupByTimeComponent));
+      
groupByTimeComponent.setSlidingStep(DateTimeUtils.constructTimeDuration(ctx.step.getText()));
+      groupByTimeComponent.setOriginalSlidingStep(ctx.step.getText());
     } else {
       groupByTimeComponent.setSlidingStep(groupByTimeComponent.getInterval());
-      
groupByTimeComponent.setSlidingStepByMonth(groupByTimeComponent.isIntervalByMonth());
+      
groupByTimeComponent.setOriginalSlidingStep(groupByTimeComponent.getOriginalInterval());
+    }
+    TimeDuration slidingStep = groupByTimeComponent.getSlidingStep();
+    if (slidingStep.containsMonth()
+        && Math.ceil(
+                ((groupByTimeComponent.getEndTime() - 
groupByTimeComponent.getStartTime())
+                    / (double) slidingStep.getMinTotalDuration(currPrecision)))
+            >= 10000) {
+      throw new SemanticException("The time windows may exceed 10000, please 
ensure your input.");
+    }
+    if (groupByTimeComponent.getSlidingStep().monthDuration == 0
+        && groupByTimeComponent.getSlidingStep().nonMonthDuration == 0) {
+      throw new SemanticException(
+          "The third parameter time slidingStep should be a positive 
integer.");
     }
-
     return groupByTimeComponent;
   }
 
@@ -1504,24 +1521,6 @@ public class ASTVisitor extends 
IoTDBSqlParserBaseVisitor<Statement> {
     }
   }
 
-  /**
-   * parse time interval or sliding step in group by query.
-   *
-   * @param duration represent duration string like: 12d8m9ns, 1y1d, etc.
-   * @return time in milliseconds, microseconds, or nanoseconds depending on 
the profile
-   */
-  private long parseTimeIntervalOrSlidingStep(
-      String duration, boolean isParsingTimeInterval, GroupByTimeComponent 
groupByTimeComponent) {
-    if (duration.toLowerCase().contains("y") || 
duration.toLowerCase().contains("mo")) {
-      if (isParsingTimeInterval) {
-        groupByTimeComponent.setIntervalByMonth(true);
-      } else {
-        groupByTimeComponent.setSlidingStepByMonth(true);
-      }
-    }
-    return DateTimeUtils.convertDurationStrToLong(duration, true);
-  }
-
   private GroupByComponent parseGroupByClause(
       GroupByAttributeClauseContext ctx, WindowType windowType) {
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGenerator.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGenerator.java
index 66b99fe4e10..4d9b084db76 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGenerator.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGenerator.java
@@ -93,6 +93,7 @@ import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.antlr.v4.runtime.CharStream;
 import org.antlr.v4.runtime.CharStreams;
@@ -244,11 +245,11 @@ public class StatementGenerator {
       GroupByTimeComponent groupByTimeComponent = new GroupByTimeComponent();
       groupByTimeComponent.setStartTime(req.getStartTime());
       groupByTimeComponent.setEndTime(req.getEndTime());
-      groupByTimeComponent.setInterval(req.getInterval());
+      groupByTimeComponent.setInterval(new TimeDuration(0, req.getInterval()));
       if (req.isSetSlidingStep()) {
-        groupByTimeComponent.setSlidingStep(req.getSlidingStep());
+        groupByTimeComponent.setSlidingStep(new TimeDuration(0, 
req.getSlidingStep()));
       } else {
-        groupByTimeComponent.setSlidingStep(req.getInterval());
+        
groupByTimeComponent.setSlidingStep(groupByTimeComponent.getInterval());
       }
       queryStatement.setGroupByTimeComponent(groupByTimeComponent);
     } else if (req.isSetStartTime()) {
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/parameter/GroupByTimeParameter.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/parameter/GroupByTimeParameter.java
index 624110c0ce9..75b39537a4b 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/parameter/GroupByTimeParameter.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/parameter/GroupByTimeParameter.java
@@ -20,7 +20,9 @@
 package org.apache.iotdb.db.queryengine.plan.planner.plan.parameter;
 
 import 
org.apache.iotdb.db.queryengine.plan.statement.component.GroupByTimeComponent;
+import org.apache.iotdb.db.utils.TimestampPrecisionUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -40,39 +42,26 @@ public class GroupByTimeParameter {
   private long endTime;
 
   // time interval
-  private long interval;
+  private TimeDuration interval;
 
   // sliding step
-  private long slidingStep;
-
-  // if it is expressed by natural month. eg, 1mo
-  private boolean isIntervalByMonth = false;
-  private boolean isSlidingStepByMonth = false;
+  private TimeDuration slidingStep;
 
   // if it is left close and right open interval
   private boolean leftCRightO;
 
   public GroupByTimeParameter() {}
 
-  public GroupByTimeParameter(
-      long startTime, long endTime, long interval, long slidingStep, boolean 
leftCRightO) {
-    this(startTime, endTime, interval, slidingStep, false, false, leftCRightO);
-  }
-
   public GroupByTimeParameter(
       long startTime,
       long endTime,
-      long interval,
-      long slidingStep,
-      boolean isIntervalByMonth,
-      boolean isSlidingStepByMonth,
+      TimeDuration interval,
+      TimeDuration slidingStep,
       boolean leftCRightO) {
     this.startTime = startTime;
     this.endTime = endTime;
     this.interval = interval;
     this.slidingStep = slidingStep;
-    this.isIntervalByMonth = isIntervalByMonth;
-    this.isSlidingStepByMonth = isSlidingStepByMonth;
     this.leftCRightO = leftCRightO;
   }
 
@@ -81,8 +70,6 @@ public class GroupByTimeParameter {
     this.endTime = groupByTimeComponent.getEndTime();
     this.interval = groupByTimeComponent.getInterval();
     this.slidingStep = groupByTimeComponent.getSlidingStep();
-    this.isIntervalByMonth = groupByTimeComponent.isIntervalByMonth();
-    this.isSlidingStepByMonth = groupByTimeComponent.isSlidingStepByMonth();
     this.leftCRightO = groupByTimeComponent.isLeftCRightO();
   }
 
@@ -102,38 +89,22 @@ public class GroupByTimeParameter {
     this.endTime = endTime;
   }
 
-  public long getInterval() {
+  public TimeDuration getInterval() {
     return interval;
   }
 
-  public void setInterval(long interval) {
+  public void setInterval(TimeDuration interval) {
     this.interval = interval;
   }
 
-  public long getSlidingStep() {
+  public TimeDuration getSlidingStep() {
     return slidingStep;
   }
 
-  public void setSlidingStep(long slidingStep) {
+  public void setSlidingStep(TimeDuration slidingStep) {
     this.slidingStep = slidingStep;
   }
 
-  public boolean isIntervalByMonth() {
-    return isIntervalByMonth;
-  }
-
-  public void setIntervalByMonth(boolean intervalByMonth) {
-    isIntervalByMonth = intervalByMonth;
-  }
-
-  public boolean isSlidingStepByMonth() {
-    return isSlidingStepByMonth;
-  }
-
-  public void setSlidingStepByMonth(boolean slidingStepByMonth) {
-    isSlidingStepByMonth = slidingStepByMonth;
-  }
-
   public boolean isLeftCRightO() {
     return leftCRightO;
   }
@@ -143,26 +114,23 @@ public class GroupByTimeParameter {
   }
 
   public boolean hasOverlap() {
-    return interval > slidingStep;
+    return interval.getTotalDuration(TimestampPrecisionUtils.currPrecision)
+        > slidingStep.getTotalDuration(TimestampPrecisionUtils.currPrecision);
   }
 
   public void serialize(ByteBuffer buffer) {
     ReadWriteIOUtils.write(startTime, buffer);
     ReadWriteIOUtils.write(endTime, buffer);
-    ReadWriteIOUtils.write(interval, buffer);
-    ReadWriteIOUtils.write(slidingStep, buffer);
-    ReadWriteIOUtils.write(isIntervalByMonth, buffer);
-    ReadWriteIOUtils.write(isSlidingStepByMonth, buffer);
+    interval.serialize(buffer);
+    slidingStep.serialize(buffer);
     ReadWriteIOUtils.write(leftCRightO, buffer);
   }
 
   public void serialize(DataOutputStream stream) throws IOException {
     ReadWriteIOUtils.write(startTime, stream);
     ReadWriteIOUtils.write(endTime, stream);
-    ReadWriteIOUtils.write(interval, stream);
-    ReadWriteIOUtils.write(slidingStep, stream);
-    ReadWriteIOUtils.write(isIntervalByMonth, stream);
-    ReadWriteIOUtils.write(isSlidingStepByMonth, stream);
+    interval.serialize(stream);
+    slidingStep.serialize(stream);
     ReadWriteIOUtils.write(leftCRightO, stream);
   }
 
@@ -170,10 +138,8 @@ public class GroupByTimeParameter {
     GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter();
     groupByTimeParameter.setStartTime(ReadWriteIOUtils.readLong(buffer));
     groupByTimeParameter.setEndTime(ReadWriteIOUtils.readLong(buffer));
-    groupByTimeParameter.setInterval(ReadWriteIOUtils.readLong(buffer));
-    groupByTimeParameter.setSlidingStep(ReadWriteIOUtils.readLong(buffer));
-    groupByTimeParameter.setIntervalByMonth(ReadWriteIOUtils.readBool(buffer));
-    
groupByTimeParameter.setSlidingStepByMonth(ReadWriteIOUtils.readBool(buffer));
+    groupByTimeParameter.setInterval(TimeDuration.deserialize(buffer));
+    groupByTimeParameter.setSlidingStep(TimeDuration.deserialize(buffer));
     groupByTimeParameter.setLeftCRightO(ReadWriteIOUtils.readBool(buffer));
     return groupByTimeParameter;
   }
@@ -186,22 +152,13 @@ public class GroupByTimeParameter {
     GroupByTimeParameter other = (GroupByTimeParameter) obj;
     return this.startTime == other.startTime
         && this.endTime == other.endTime
-        && this.interval == other.interval
-        && this.slidingStep == other.slidingStep
-        && this.isSlidingStepByMonth == other.isSlidingStepByMonth
-        && this.isIntervalByMonth == other.isIntervalByMonth
+        && this.interval.equals(other.interval)
+        && this.slidingStep.equals(other.slidingStep)
         && this.leftCRightO == other.leftCRightO;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(
-        startTime,
-        endTime,
-        interval,
-        slidingStep,
-        isIntervalByMonth,
-        isSlidingStepByMonth,
-        leftCRightO);
+    return Objects.hash(startTime, endTime, interval, slidingStep, 
leftCRightO);
   }
 }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/GroupByTimeComponent.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/GroupByTimeComponent.java
index 25deeabbbb0..d2555b5af9f 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/GroupByTimeComponent.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/GroupByTimeComponent.java
@@ -19,8 +19,8 @@
 
 package org.apache.iotdb.db.queryengine.plan.statement.component;
 
-import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.db.queryengine.plan.statement.StatementNode;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 /** This class maintains information of {@code GROUP BY} clause. */
 public class GroupByTimeComponent extends StatementNode {
@@ -30,14 +30,14 @@ public class GroupByTimeComponent extends StatementNode {
   private long endTime;
 
   // time interval
-  private long interval;
+  private TimeDuration interval;
 
   // sliding step
-  private long slidingStep;
+  private TimeDuration slidingStep;
 
-  // if it is expressed by natural month. eg, 1mo
-  private boolean isIntervalByMonth = false;
-  private boolean isSlidingStepByMonth = false;
+  private String originalInterval;
+
+  private String originalSlidingStep;
 
   // if it is left close and right open interval
   private boolean leftCRightO = true;
@@ -54,11 +54,11 @@ public class GroupByTimeComponent extends StatementNode {
     this.leftCRightO = leftCRightO;
   }
 
-  public long getInterval() {
+  public TimeDuration getInterval() {
     return interval;
   }
 
-  public void setInterval(long interval) {
+  public void setInterval(TimeDuration interval) {
     this.interval = interval;
   }
 
@@ -78,28 +78,24 @@ public class GroupByTimeComponent extends StatementNode {
     this.endTime = endTime;
   }
 
-  public long getSlidingStep() {
+  public TimeDuration getSlidingStep() {
     return slidingStep;
   }
 
-  public void setSlidingStep(long slidingStep) {
+  public void setSlidingStep(TimeDuration slidingStep) {
     this.slidingStep = slidingStep;
   }
 
-  public boolean isSlidingStepByMonth() {
-    return isSlidingStepByMonth;
-  }
-
-  public void setSlidingStepByMonth(boolean isSlidingStepByMonth) {
-    this.isSlidingStepByMonth = isSlidingStepByMonth;
+  public String getOriginalInterval() {
+    return originalInterval;
   }
 
-  public boolean isIntervalByMonth() {
-    return isIntervalByMonth;
+  public void setOriginalInterval(String originalInterval) {
+    this.originalInterval = originalInterval;
   }
 
-  public void setIntervalByMonth(boolean isIntervalByMonth) {
-    this.isIntervalByMonth = isIntervalByMonth;
+  public void setOriginalSlidingStep(String originalSlidingStep) {
+    this.originalSlidingStep = originalSlidingStep;
   }
 
   public String toSQLString() {
@@ -126,20 +122,10 @@ public class GroupByTimeComponent extends StatementNode {
       }
       sqlBuilder.append(',').append(' ');
     }
-    String intervalStr =
-        interval
-            + (isIntervalByMonth
-                ? "mo"
-                : 
CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
-    String slidingStepStr =
-        slidingStep
-            + (isSlidingStepByMonth
-                ? "mo"
-                : 
CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
-    sqlBuilder.append(intervalStr);
-    if (!slidingStepStr.equals(intervalStr)) {
+    sqlBuilder.append(originalInterval);
+    if (!originalSlidingStep.equals(originalInterval)) {
       sqlBuilder.append(',').append(' ');
-      sqlBuilder.append(slidingStepStr);
+      sqlBuilder.append(originalSlidingStep);
     }
     sqlBuilder.append(')');
     return sqlBuilder.toString();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
index 2bc5afddcdc..48befee79cd 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.protocol.session.SessionManager;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import java.time.DateTimeException;
 import java.time.Duration;
@@ -764,23 +765,66 @@ public class DateTimeUtils {
 
   public static final long MS_TO_MONTH = 30 * 86400_000L;
 
+  public static long calcPositiveIntervalByMonth(
+      long startTime, TimeDuration duration, long times) {
+    return TimeDuration.calcPositiveIntervalByMonth(
+        startTime,
+        duration,
+        times,
+        SessionManager.getInstance().getSessionTimeZone(),
+        TimestampPrecisionUtils.currPrecision);
+  }
+
+  public static long calcNegativeIntervalByMonth(long startTime, TimeDuration 
duration) {
+    return TimeDuration.calcNegativeIntervalByMonth(
+        startTime,
+        duration,
+        SessionManager.getInstance().getSessionTimeZone(),
+        TimestampPrecisionUtils.currPrecision);
+  }
+
   /**
-   * add natural months based on the startTime to avoid edge cases, ie 2/28
+   * Storage the duration into two parts: month part and non-month part, the 
non-month part's
+   * precision is depended on current time precision. e.g. ms precision: 
'1y1mo1ms' -> monthDuration
+   * = 13, nonMonthDuration = 1, ns precision: '1y1mo1ms' -> monthDuration = 
13, nonMonthDuration =
+   * 1000_000.
    *
-   * @param startTime current start time
-   * @param numMonths numMonths is updated in hasNextWithoutConstraint()
-   * @return nextStartTime
+   * @param duration the input duration string
+   * @return the TimeDuration instance contains month part and non-month part
    */
-  public static long calcIntervalByMonth(long startTime, long numMonths) {
-    Calendar calendar = Calendar.getInstance();
-    calendar.setTimeZone(SessionManager.getInstance().getSessionTimeZone());
-    calendar.setTimeInMillis(startTime);
-    boolean isLastDayOfMonth =
-        calendar.get(Calendar.DAY_OF_MONTH) == 
calendar.getActualMaximum(Calendar.DAY_OF_MONTH);
-    calendar.add(Calendar.MONTH, (int) (numMonths));
-    if (isLastDayOfMonth) {
-      calendar.set(Calendar.DAY_OF_MONTH, 
calendar.getActualMaximum(Calendar.DAY_OF_MONTH));
+  public static TimeDuration constructTimeDuration(String duration) {
+    duration = duration.toLowerCase();
+    String currTimePrecision = 
CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
+    int temp = 0;
+    int monthDuration = 0;
+    long nonMonthDuration = 0;
+    for (int i = 0; i < duration.length(); i++) {
+      char ch = duration.charAt(i);
+      if (Character.isDigit(ch)) {
+        temp *= 10;
+        temp += (ch - '0');
+      } else {
+        String unit = String.valueOf(duration.charAt(i));
+        // This is to identify units with two letters.
+        if (i + 1 < duration.length() && !Character.isDigit(duration.charAt(i 
+ 1))) {
+          i++;
+          unit += duration.charAt(i);
+        }
+        if (unit.equals("y")) {
+          monthDuration += temp * 12;
+          temp = 0;
+          continue;
+        }
+        if (unit.equals("mo")) {
+          monthDuration += temp;
+          temp = 0;
+          continue;
+        }
+        nonMonthDuration +=
+            DateTimeUtils.convertDurationStrToLong(-1, temp, unit, 
currTimePrecision);
+        temp = 0;
+      }
     }
-    return calendar.getTimeInMillis();
+    return new TimeDuration(monthDuration, nonMonthDuration);
   }
 }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimestampPrecisionUtils.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimestampPrecisionUtils.java
index c17a85cc2bc..e57f2e851c5 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimestampPrecisionUtils.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/TimestampPrecisionUtils.java
@@ -26,7 +26,7 @@ import java.util.concurrent.TimeUnit;
 public class TimestampPrecisionUtils {
   static String TIMESTAMP_PRECISION =
       CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
-  private static boolean isTimestampPrecisionCheckEnabled =
+  private static final boolean isTimestampPrecisionCheckEnabled =
       
CommonDescriptor.getInstance().getConfig().isTimestampPrecisionCheckEnabled();
 
   @FunctionalInterface
@@ -34,28 +34,33 @@ public class TimestampPrecisionUtils {
     R apply(T1 t1, T2 t2);
   }
 
-  private static final ConvertFunction<Long, TimeUnit, Long> convertFunction;
+  private static final ConvertFunction<Long, TimeUnit, Long> 
convertToCurrPrecisionFunction;
+  public static final TimeUnit currPrecision;
 
   static {
     switch (TIMESTAMP_PRECISION) {
       case "ms":
-        convertFunction = TimeUnit.MILLISECONDS::convert;
+        convertToCurrPrecisionFunction = TimeUnit.MILLISECONDS::convert;
+        currPrecision = TimeUnit.MILLISECONDS;
         break;
       case "us":
-        convertFunction = TimeUnit.MICROSECONDS::convert;
+        convertToCurrPrecisionFunction = TimeUnit.MICROSECONDS::convert;
+        currPrecision = TimeUnit.MICROSECONDS;
         break;
       case "ns":
-        convertFunction = TimeUnit.NANOSECONDS::convert;
+        convertToCurrPrecisionFunction = TimeUnit.NANOSECONDS::convert;
+        currPrecision = TimeUnit.NANOSECONDS;
         break;
         // this case will never reach
       default:
-        convertFunction = null;
+        throw new UnsupportedOperationException(
+            "not supported time_precision: " + TIMESTAMP_PRECISION);
     }
   }
 
   /** convert specific precision timestamp to current precision timestamp */
   public static long convertToCurrPrecision(long sourceTime, TimeUnit 
sourceUnit) {
-    return convertFunction.apply(sourceTime, sourceUnit);
+    return convertToCurrPrecisionFunction.apply(sourceTime, sourceUnit);
   }
 
   /** check whether the input timestamp match the current system timestamp 
precision. */
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/aggregation/TimeRangeIteratorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/aggregation/TimeRangeIteratorTest.java
index 7a26f412dc5..592110d41fc 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/aggregation/TimeRangeIteratorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/aggregation/TimeRangeIteratorTest.java
@@ -22,13 +22,15 @@ package 
org.apache.iotdb.db.queryengine.execution.aggregation;
 import 
org.apache.iotdb.db.queryengine.execution.aggregation.timerangeiterator.ITimeRangeIterator;
 import 
org.apache.iotdb.db.queryengine.execution.aggregation.timerangeiterator.TimeRangeIteratorFactory;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-public class TimeRangeIteratorTest {
+import java.sql.Timestamp;
 
-  private static final long MS_TO_MONTH = 30 * 86400_000L;
+public class TimeRangeIteratorTest {
+  private static final long MS_TO_DAY = 86400_000L;
 
   @Test
   public void testNotSplitTimeRange() {
@@ -46,17 +48,18 @@ public class TimeRangeIteratorTest {
       "[ 30 : 31 ]"
     };
 
-    long startTime = 0, endTime = 32, interval = 4, slidingStep = 3;
+    long startTime = 0, endTime = 32;
+    TimeDuration interval = new TimeDuration(0, 4), slidingStep = new 
TimeDuration(0, 3);
 
     ITimeRangeIterator timeRangeIterator =
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            startTime, endTime, interval, slidingStep, true, false, false, 
true, false);
+            startTime, endTime, interval, slidingStep, true, true, false);
 
     checkRes(timeRangeIterator, res);
 
     ITimeRangeIterator descTimeRangeIterator =
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            startTime, endTime, interval, slidingStep, false, false, false, 
true, false);
+            startTime, endTime, interval, slidingStep, false, true, false);
 
     checkRes(descTimeRangeIterator, res);
   }
@@ -162,41 +165,54 @@ public class TimeRangeIteratorTest {
       "[ 0 : 3 ]", "[ 6 : 9 ]", "[ 12 : 15 ]", "[ 18 : 21 ]", "[ 24 : 27 ]", 
"[ 30 : 31 ]"
     };
 
+    TimeDuration interval = new TimeDuration(0, 4);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 1, true, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 1), true, true, true),
         res4_1);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 2, true, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 2), true, true, true),
         res4_2);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 3, true, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 3), true, true, true),
         res4_3);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 4, true, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 4), true, true, true),
         res4_4);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 5, true, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 5), true, true, true),
         res4_5);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 6, true, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 6), true, true, true),
         res4_6);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 1, false, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 1), false, true, true),
         res4_1);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 2, false, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 2), false, true, true),
         res4_2);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 3, false, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 3), false, true, true),
         res4_3);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 4, false, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 4), false, true, true),
         res4_4);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 5, false, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 5), false, true, true),
         res4_5);
     checkRes(
-        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 6, false, 
false, false, true, true),
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            0, 32, interval, new TimeDuration(0, 6), false, true, true),
         res4_6);
   }
 
@@ -264,36 +280,151 @@ public class TimeRangeIteratorTest {
         TimeRangeIteratorFactory.getTimeRangeIterator(
             1604102400000L,
             1617148800000L,
-            MS_TO_MONTH,
-            MS_TO_MONTH,
-            true,
-            true,
+            new TimeDuration(1, 0),
+            new TimeDuration(1, 0),
             true,
             true,
             false),
         res1);
     checkRes(
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            1604102400000L, 1617148800000L, MS_TO_MONTH, MS_TO_MONTH, true, 
true, true, true, true),
+            1604102400000L,
+            1617148800000L,
+            new TimeDuration(1, 0),
+            new TimeDuration(1, 0),
+            true,
+            true,
+            true),
         res1);
     checkRes(
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            1604102400000L, 1617148800000L, 864000000, MS_TO_MONTH, true, 
false, true, true, false),
+            1604102400000L,
+            1617148800000L,
+            new TimeDuration(0, 10 * MS_TO_DAY),
+            new TimeDuration(1, 0),
+            true,
+            true,
+            false),
         res2);
     checkRes(
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            1604102400000L, 1617148800000L, 864000000, MS_TO_MONTH, true, 
false, true, true, true),
+            1604102400000L,
+            1617148800000L,
+            new TimeDuration(0, 10 * MS_TO_DAY),
+            new TimeDuration(1, 0),
+            true,
+            true,
+            true),
         res2);
     checkRes(
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            1604102400000L, 1617148800000L, MS_TO_MONTH, 864000000, true, 
true, false, true, false),
+            1604102400000L,
+            1617148800000L,
+            new TimeDuration(1, 0),
+            new TimeDuration(0, 10 * MS_TO_DAY),
+            true,
+            true,
+            false),
         res3);
     checkRes(
         TimeRangeIteratorFactory.getTimeRangeIterator(
-            1604102400000L, 1617148800000L, MS_TO_MONTH, 864000000, true, 
true, false, true, true),
+            1604102400000L,
+            1617148800000L,
+            new TimeDuration(1, 0),
+            new TimeDuration(0, 10 * MS_TO_DAY),
+            true,
+            true,
+            true),
         res4);
   }
 
+  @Test
+  public void testMixedUnit() {
+    String[] res =
+        new String[] {
+          "[ "
+              + Timestamp.valueOf("2023-01-28 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-03-01 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-03-01 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-04-02 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-04-02 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-05-03 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-05-03 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-05-29 00:00:00").getTime() - 1L)
+              + " ]"
+        };
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            Timestamp.valueOf("2023-01-28 00:00:00").getTime(),
+            Timestamp.valueOf("2023-05-29 00:00:00").getTime(),
+            new TimeDuration(1, MS_TO_DAY),
+            new TimeDuration(1, MS_TO_DAY),
+            true,
+            true,
+            true),
+        res);
+
+    res =
+        new String[] {
+          "[ "
+              + Timestamp.valueOf("2023-01-28 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-03-01 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-03-01 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-03-02 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-03-02 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-04-02 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-04-02 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-04-03 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-04-03 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-05-03 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-05-03 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-05-04 00:00:00").getTime() - 1L)
+              + " ]",
+          "[ "
+              + Timestamp.valueOf("2023-05-04 00:00:00").getTime()
+              + " : "
+              + (Timestamp.valueOf("2023-05-29 00:00:00").getTime() - 1L)
+              + " ]",
+        };
+    // slidingStep has overlap with interval
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            Timestamp.valueOf("2023-01-28 00:00:00").getTime(),
+            Timestamp.valueOf("2023-05-29 00:00:00").getTime(),
+            new TimeDuration(1, 2 * MS_TO_DAY),
+            new TimeDuration(1, MS_TO_DAY),
+            true,
+            true,
+            true),
+        res);
+  }
+
   private void checkRes(ITimeRangeIterator timeRangeIterator, String[] res) {
     Assert.assertEquals(res.length, timeRangeIterator.getTotalIntervalNum());
 
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationOperatorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationOperatorTest.java
index 026d4925444..78d0185e299 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationOperatorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AggregationOperatorTest.java
@@ -48,6 +48,7 @@ import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import com.google.common.util.concurrent.ListenableFuture;
@@ -189,7 +190,8 @@ public class AggregationOperatorTest {
           {20099, 20199, 10259, 10379},
           {20000, 20100, 260, 380}
         };
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<TAggregationType> aggregationTypes = new ArrayList<>();
     aggregationTypes.add(TAggregationType.COUNT);
     aggregationTypes.add(TAggregationType.SUM);
@@ -245,7 +247,8 @@ public class AggregationOperatorTest {
     aggregationTypes.add(TAggregationType.AVG);
     aggregationTypes.add(TAggregationType.FIRST_VALUE);
     aggregationTypes.add(TAggregationType.LAST_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<List<InputLocation[]>> inputLocations = new ArrayList<>();
     for (int i = 0; i < aggregationTypes.size(); i++) {
       List<InputLocation[]> inputLocationForOneAggregator = new ArrayList<>();
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
index 89d859a1004..0426f0e7704 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java
@@ -49,6 +49,7 @@ import org.apache.iotdb.tsfile.read.common.block.TsBlock;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
@@ -419,7 +420,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
   @Test
   public void testGroupByWithoutGlobalTimeFilter() throws Exception {
     int[] result = new int[] {100, 100, 100, 99};
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<Aggregator> aggregators = new ArrayList<>();
     for (int i = 0; i < measurementSchemas.size(); i++) {
       TSDataType dataType = measurementSchemas.get(i).getType();
@@ -458,7 +460,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
   public void testGroupByWithGlobalTimeFilter() throws Exception {
     int[] result = new int[] {0, 80, 100, 80};
     Filter timeFilter = new AndFilter(TimeFilter.gtEq(120), 
TimeFilter.ltEq(379));
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<Aggregator> aggregators = new ArrayList<>();
     for (int i = 0; i < measurementSchemas.size(); i++) {
       TSDataType dataType = measurementSchemas.get(i).getType();
@@ -507,7 +510,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
     aggregationTypes.add(TAggregationType.LAST_VALUE);
     aggregationTypes.add(TAggregationType.MAX_VALUE);
     aggregationTypes.add(TAggregationType.MIN_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<Aggregator> aggregators = new ArrayList<>();
     List<InputLocation[]> inputLocations =
         Collections.singletonList(new InputLocation[] {new InputLocation(0, 
1)});
@@ -550,7 +554,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
     aggregationTypes.add(TAggregationType.LAST_VALUE);
     aggregationTypes.add(TAggregationType.MAX_VALUE);
     aggregationTypes.add(TAggregationType.MIN_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<Aggregator> aggregators = new ArrayList<>();
     List<InputLocation[]> inputLocations =
         Collections.singletonList(new InputLocation[] {new InputLocation(0, 
1)});
@@ -582,7 +587,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
   @Test
   public void testGroupBySlidingTimeWindow() throws Exception {
     int[] result = new int[] {50, 50, 50, 50, 50, 50, 50, 49};
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 50, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 50), true);
     List<TAggregationType> aggregationTypes = 
Collections.singletonList(TAggregationType.COUNT);
     List<Aggregator> aggregators = new ArrayList<>();
     List<InputLocation[]> inputLocations =
@@ -613,7 +619,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
   public void testGroupBySlidingTimeWindow2() throws Exception {
     int[] timeColumn = new int[] {0, 20, 30, 50, 60, 80, 90, 110, 120, 140};
     int[] result = new int[] {20, 10, 20, 10, 20, 10, 20, 10, 20, 9};
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
149, 50, 30, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 149, new TimeDuration(0, 50), new 
TimeDuration(0, 30), true);
     List<TAggregationType> aggregationTypes = 
Collections.singletonList(TAggregationType.COUNT);
     List<Aggregator> aggregators = new ArrayList<>();
     List<InputLocation[]> inputLocations =
@@ -655,7 +662,8 @@ public class AlignedSeriesAggregationScanOperatorTest {
     aggregationTypes.add(TAggregationType.LAST_VALUE);
     aggregationTypes.add(TAggregationType.MAX_VALUE);
     aggregationTypes.add(TAggregationType.MIN_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
149, 50, 30, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 149, new TimeDuration(0, 50), new 
TimeDuration(0, 30), true);
     List<Aggregator> aggregators = new ArrayList<>();
     List<InputLocation[]> inputLocations =
         Collections.singletonList(new InputLocation[] {new InputLocation(0, 
1)});
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java
index 5bc2fc19639..d4f92fbcea4 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java
@@ -45,6 +45,7 @@ import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import io.airlift.units.Duration;
@@ -125,7 +126,8 @@ public class HorizontallyConcatOperatorTest {
               HORIZONTALLY_CONCAT_OPERATOR_TEST_SG + ".device0.sensor0", 
TSDataType.INT32);
       List<TAggregationType> aggregationTypes =
           Arrays.asList(TAggregationType.COUNT, TAggregationType.SUM, 
TAggregationType.FIRST_VALUE);
-      GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
10, 1, 1, true);
+      GroupByTimeParameter groupByTimeParameter =
+          new GroupByTimeParameter(0, 10, new TimeDuration(0, 1), new 
TimeDuration(0, 1), true);
       List<Aggregator> aggregators = new ArrayList<>();
       AccumulatorFactory.createAccumulators(
               aggregationTypes,
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java
index 41dbc268239..4ebf272eee9 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorMemoryTest.java
@@ -95,6 +95,7 @@ import 
org.apache.iotdb.tsfile.read.common.block.column.TimeColumn;
 import org.apache.iotdb.tsfile.read.common.type.BooleanType;
 import org.apache.iotdb.tsfile.read.common.type.LongType;
 import org.apache.iotdb.tsfile.read.common.type.TypeEnum;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import com.google.common.collect.Sets;
 import org.junit.Test;
@@ -1041,8 +1042,8 @@ public class OperatorMemoryTest {
           new GroupByTimeParameter(
               0,
               2 * maxTsBlockLineNumber,
-              maxTsBlockLineNumber / 100,
-              maxTsBlockLineNumber / 100,
+              new TimeDuration(0, maxTsBlockLineNumber / 100),
+              new TimeDuration(0, maxTsBlockLineNumber / 100),
               true);
       List<AggregationDescriptor> aggregationDescriptors3 =
           Arrays.asList(
@@ -1079,7 +1080,9 @@ public class OperatorMemoryTest {
           
seriesAggregationScanOperator3.calculateRetainedSizeAfterCallingNext());
 
       // case4: with group by, total window num > 1000
-      groupByTimeParameter = new GroupByTimeParameter(0, 2 * 
maxTsBlockLineNumber, 1, 1, true);
+      groupByTimeParameter =
+          new GroupByTimeParameter(
+              0, 2 * maxTsBlockLineNumber, new TimeDuration(0, 1), new 
TimeDuration(0, 1), true);
       List<AggregationDescriptor> aggregationDescriptors4 =
           Arrays.asList(
               new AggregationDescriptor(
@@ -1117,7 +1120,9 @@ public class OperatorMemoryTest {
           
seriesAggregationScanOperator4.calculateRetainedSizeAfterCallingNext());
 
       // case5: over DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES
-      groupByTimeParameter = new GroupByTimeParameter(0, 2 * 
maxTsBlockLineNumber, 1, 1, true);
+      groupByTimeParameter =
+          new GroupByTimeParameter(
+              0, 2 * maxTsBlockLineNumber, new TimeDuration(0, 1), new 
TimeDuration(0, 1), true);
       List<AggregationDescriptor> aggregationDescriptors5 =
           Arrays.asList(
               new AggregationDescriptor(
@@ -1246,7 +1251,8 @@ public class OperatorMemoryTest {
                         true),
                     o.getStep())));
 
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
1000, 10, 10, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 1000, new TimeDuration(0, 10), new 
TimeDuration(0, 10), true);
     ITimeRangeIterator timeRangeIterator = 
initTimeRangeIterator(groupByTimeParameter, true, false);
     long maxReturnSize =
         AggregationUtil.calculateMaxAggregationResultSize(
@@ -1318,7 +1324,8 @@ public class OperatorMemoryTest {
                         true),
                     o.getStep())));
 
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
1000, 10, 5, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 1000, new TimeDuration(0, 10), new 
TimeDuration(0, 5), true);
     ITimeRangeIterator timeRangeIterator = 
initTimeRangeIterator(groupByTimeParameter, true, false);
     long maxReturnSize =
         AggregationUtil.calculateMaxAggregationResultSize(
@@ -1396,7 +1403,8 @@ public class OperatorMemoryTest {
                         true),
                     o.getStep())));
 
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
1000, 10, 10, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 1000, new TimeDuration(0, 10), new 
TimeDuration(0, 10), true);
     ITimeRangeIterator timeRangeIterator = 
initTimeRangeIterator(groupByTimeParameter, true, false);
     long maxReturnSize =
         AggregationUtil.calculateMaxAggregationResultSize(
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/RawDataAggregationOperatorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/RawDataAggregationOperatorTest.java
index 5ea3fbd23ba..a3c4b395e48 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/RawDataAggregationOperatorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/RawDataAggregationOperatorTest.java
@@ -59,6 +59,7 @@ import org.apache.iotdb.tsfile.read.common.block.TsBlock;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterType;
 import org.apache.iotdb.tsfile.read.filter.operator.Gt;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.junit.After;
@@ -244,7 +245,8 @@ public class RawDataAggregationOperatorTest {
         inputLocations.add(inputLocationForOneAggregator);
       }
     }
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
 
     WindowParameter windowParameter = new TimeWindowParameter(false);
 
@@ -304,7 +306,8 @@ public class RawDataAggregationOperatorTest {
       inputLocationForOneAggregator.add(new InputLocation[] {new 
InputLocation(0, i)});
       inputLocations.add(inputLocationForOneAggregator);
     }
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
 
     WindowParameter windowParameter = new TimeWindowParameter(false);
 
@@ -368,7 +371,8 @@ public class RawDataAggregationOperatorTest {
         inputLocations.add(inputLocationForOneAggregator);
       }
     }
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
 
     WindowParameter windowParameter = new TimeWindowParameter(true);
 
@@ -433,7 +437,8 @@ public class RawDataAggregationOperatorTest {
         inputLocations.add(inputLocationForOneAggregator);
       }
     }
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
600, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 600, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
 
     WindowParameter windowParameter = new TimeWindowParameter(true);
 
@@ -501,7 +506,8 @@ public class RawDataAggregationOperatorTest {
         inputLocations.add(inputLocationForOneAggregator);
       }
     }
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
600, 100, 100, false);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 600, new TimeDuration(0, 100), new 
TimeDuration(0, 100), false);
 
     WindowParameter windowParameter = new TimeWindowParameter(false);
 
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java
index 0de9dcd054e..2081872ac7a 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java
@@ -47,6 +47,7 @@ import org.apache.iotdb.tsfile.read.common.block.TsBlock;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import com.google.common.collect.Sets;
@@ -352,7 +353,8 @@ public class SeriesAggregationScanOperatorTest {
   @Test
   public void testGroupByWithoutGlobalTimeFilter() throws Exception {
     int[] result = new int[] {100, 100, 100, 99};
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<TAggregationType> aggregationTypes = 
Collections.singletonList(TAggregationType.COUNT);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
@@ -383,7 +385,8 @@ public class SeriesAggregationScanOperatorTest {
   public void testGroupByWithGlobalTimeFilter() throws Exception {
     int[] result = new int[] {0, 80, 100, 80};
     Filter timeFilter = new AndFilter(TimeFilter.gtEq(120), 
TimeFilter.ltEq(379));
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<TAggregationType> aggregationTypes = 
Collections.singletonList(TAggregationType.COUNT);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
@@ -424,7 +427,8 @@ public class SeriesAggregationScanOperatorTest {
     aggregationTypes.add(TAggregationType.LAST_VALUE);
     aggregationTypes.add(TAggregationType.MAX_VALUE);
     aggregationTypes.add(TAggregationType.MIN_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
             aggregationTypes,
@@ -467,7 +471,8 @@ public class SeriesAggregationScanOperatorTest {
     aggregationTypes.add(TAggregationType.LAST_VALUE);
     aggregationTypes.add(TAggregationType.MAX_VALUE);
     aggregationTypes.add(TAggregationType.MIN_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 100, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 100), true);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
             aggregationTypes,
@@ -499,7 +504,8 @@ public class SeriesAggregationScanOperatorTest {
   @Test
   public void testGroupBySlidingTimeWindow() throws Exception {
     int[] result = new int[] {50, 50, 50, 50, 50, 50, 50, 49};
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
399, 100, 50, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 399, new TimeDuration(0, 100), new 
TimeDuration(0, 50), true);
     List<TAggregationType> aggregationTypes = 
Collections.singletonList(TAggregationType.COUNT);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
@@ -530,7 +536,8 @@ public class SeriesAggregationScanOperatorTest {
   public void testGroupBySlidingTimeWindow2() throws Exception {
     int[] timeColumn = new int[] {0, 20, 30, 50, 60, 80, 90, 110, 120, 140};
     int[] result = new int[] {20, 10, 20, 10, 20, 10, 20, 10, 20, 9};
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
149, 50, 30, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 149, new TimeDuration(0, 50), new 
TimeDuration(0, 30), true);
     List<TAggregationType> aggregationTypes = 
Collections.singletonList(TAggregationType.COUNT);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
@@ -572,7 +579,8 @@ public class SeriesAggregationScanOperatorTest {
     aggregationTypes.add(TAggregationType.LAST_VALUE);
     aggregationTypes.add(TAggregationType.MAX_VALUE);
     aggregationTypes.add(TAggregationType.MIN_VALUE);
-    GroupByTimeParameter groupByTimeParameter = new GroupByTimeParameter(0, 
149, 50, 30, true);
+    GroupByTimeParameter groupByTimeParameter =
+        new GroupByTimeParameter(0, 149, new TimeDuration(0, 50), new 
TimeDuration(0, 30), true);
     List<Aggregator> aggregators = new ArrayList<>();
     AccumulatorFactory.createAccumulators(
             aggregationTypes,
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SlidingWindowAggregationOperatorTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SlidingWindowAggregationOperatorTest.java
index 59be047c3f4..31dbb9791e9 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SlidingWindowAggregationOperatorTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SlidingWindowAggregationOperatorTest.java
@@ -48,6 +48,7 @@ import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.junit.After;
@@ -116,7 +117,7 @@ public class SlidingWindowAggregationOperatorTest {
           Collections.singletonList(Collections.singletonList(new 
InputLocation(0, 7))));
 
   private final GroupByTimeParameter groupByTimeParameter =
-      new GroupByTimeParameter(0, 300, 100, 50, true);
+      new GroupByTimeParameter(0, 300, new TimeDuration(0, 100), new 
TimeDuration(0, 50), true);
 
   private static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES =
       TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes();
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/AggregationNodeSerdeTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/AggregationNodeSerdeTest.java
index 8faa5f2e242..16ed58eebf9 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/AggregationNodeSerdeTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/AggregationNodeSerdeTest.java
@@ -34,6 +34,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.ValueFilter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import com.google.common.collect.Sets;
 import org.junit.Test;
@@ -50,7 +51,7 @@ public class AggregationNodeSerdeTest {
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException {
     GroupByTimeParameter groupByTimeParameter =
-        new GroupByTimeParameter(1, 100, 1, 1, false, false, false);
+        new GroupByTimeParameter(1, 100, new TimeDuration(0, 1), new 
TimeDuration(0, 1), false);
     SeriesAggregationScanNode seriesAggregationScanNode =
         new SeriesAggregationScanNode(
             new PlanNodeId("TestSeriesAggregateScanNode"),
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
index 25b184f5e83..fd66caf8e12 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByLevelNodeSerdeTest.java
@@ -35,6 +35,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.ValueFilter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.junit.Test;
 
@@ -52,7 +53,7 @@ public class GroupByLevelNodeSerdeTest {
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException, 
IOException {
     GroupByTimeParameter groupByTimeParameter =
-        new GroupByTimeParameter(1, 100, 1, 1, true, true, true);
+        new GroupByTimeParameter(1, 100, new TimeDuration(1, 0), new 
TimeDuration(1, 0), true);
     SeriesAggregationScanNode seriesAggregationScanNode1 =
         new SeriesAggregationScanNode(
             new PlanNodeId("TestSeriesAggregateScanNode"),
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByTagNodeSerdeTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByTagNodeSerdeTest.java
index e79e87f9f48..a5c1db56544 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByTagNodeSerdeTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/process/GroupByTagNodeSerdeTest.java
@@ -36,6 +36,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.ValueFilter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -56,7 +57,7 @@ public class GroupByTagNodeSerdeTest {
   @Test
   public void testSerializeAndDeserialize() throws IllegalPathException, 
IOException {
     GroupByTimeParameter groupByTimeParameter =
-        new GroupByTimeParameter(1, 100, 1, 1, true, true, true);
+        new GroupByTimeParameter(1, 100, new TimeDuration(1, 0), new 
TimeDuration(1, 0), true);
     CrossSeriesAggregationDescriptor s1MaxTime =
         new CrossSeriesAggregationDescriptor(
             TAggregationType.MAX_TIME.name().toLowerCase(),
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
index 3fe497ca034..f768ace4d7d 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/source/SeriesAggregationScanNodeSerdeTest.java
@@ -34,6 +34,7 @@ import 
org.apache.iotdb.db.queryengine.plan.statement.component.Ordering;
 import org.apache.iotdb.tsfile.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.ValueFilter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import com.google.common.collect.Sets;
 import org.junit.Test;
@@ -55,7 +56,7 @@ public class SeriesAggregationScanNodeSerdeTest {
             AggregationStep.FINAL,
             Collections.singletonList(new TimeSeriesOperand(new 
PartialPath("root.sg.d1.s1")))));
     GroupByTimeParameter groupByTimeParameter =
-        new GroupByTimeParameter(1, 100, 1, 1, true, true, true);
+        new GroupByTimeParameter(1, 100, new TimeDuration(1, 0), new 
TimeDuration(1, 0), true);
     SeriesAggregationScanNode seriesAggregationScanNode =
         new SeriesAggregationScanNode(
             new PlanNodeId("TestSeriesAggregateScanNode"),
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/qp/utils/DatetimeQueryDataSetUtilsTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/DateTimeUtilsTest.java
similarity index 89%
rename from 
iotdb-core/datanode/src/test/java/org/apache/iotdb/db/qp/utils/DatetimeQueryDataSetUtilsTest.java
rename to 
iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/DateTimeUtilsTest.java
index ddc6a8dabcc..a4258723271 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/qp/utils/DatetimeQueryDataSetUtilsTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/DateTimeUtilsTest.java
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.qp.utils;
+package org.apache.iotdb.db.utils;
 
-import org.apache.iotdb.db.utils.DateTimeUtils;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -29,7 +29,7 @@ import java.time.ZonedDateTime;
 
 import static org.junit.Assert.assertEquals;
 
-public class DatetimeQueryDataSetUtilsTest {
+public class DateTimeUtilsTest {
 
   private ZoneOffset zoneOffset;
   private ZoneId zoneId;
@@ -235,4 +235,31 @@ public class DatetimeQueryDataSetUtilsTest {
       assertEquals(res, DateTimeUtils.convertDatetimeStrToLong(str, zoneId));
     }
   }
+
+  @Test
+  public void testConstructTimeDuration() {
+    TimeDuration timeDuration = DateTimeUtils.constructTimeDuration("1y1d1ns");
+    Assert.assertEquals(12, timeDuration.monthDuration);
+    Assert.assertEquals(86400_000L, timeDuration.nonMonthDuration);
+
+    timeDuration = DateTimeUtils.constructTimeDuration("1y1mo1d1ms1ns");
+    Assert.assertEquals(13, timeDuration.monthDuration);
+    Assert.assertEquals(86400_001L, timeDuration.nonMonthDuration);
+
+    timeDuration = DateTimeUtils.constructTimeDuration("1d1ns");
+    Assert.assertEquals(0, timeDuration.monthDuration);
+    Assert.assertEquals(86400_000L, timeDuration.nonMonthDuration);
+
+    timeDuration = DateTimeUtils.constructTimeDuration("1y");
+    Assert.assertEquals(12, timeDuration.monthDuration);
+    Assert.assertEquals(0, timeDuration.nonMonthDuration);
+
+    timeDuration = DateTimeUtils.constructTimeDuration("1mo");
+    Assert.assertEquals(1, timeDuration.monthDuration);
+    Assert.assertEquals(0, timeDuration.nonMonthDuration);
+
+    timeDuration = DateTimeUtils.constructTimeDuration("1y1mo");
+    Assert.assertEquals(13, timeDuration.monthDuration);
+    Assert.assertEquals(0, timeDuration.nonMonthDuration);
+  }
 }
diff --git 
a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
 
b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
index 6ee54c11875..98bf4fc0332 100644
--- 
a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
+++ 
b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
@@ -47,6 +47,11 @@ public class GroupByFilter implements Filter, Serializable {
     this.endTime = endTime;
   }
 
+  public GroupByFilter(long startTime, long endTime) {
+    this.startTime = startTime;
+    this.endTime = endTime;
+  }
+
   public GroupByFilter() {}
 
   @Override
diff --git 
a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
 
b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
index 2f500b6656d..0f0acc53323 100644
--- 
a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
+++ 
b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilter.java
@@ -22,61 +22,65 @@ package org.apache.iotdb.tsfile.read.filter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterSerializeId;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Objects;
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.iotdb.tsfile.utils.TimeDuration.calcPositiveIntervalByMonth;
+import static 
org.apache.iotdb.tsfile.utils.TimeDuration.getConsecutiveTimesIntervalByMonth;
 
 /**
  * GroupByMonthFilter is used to handle natural month slidingStep and interval 
by generating
  * dynamically. Attention: it's only supported to access in ascending order 
now.
  */
 public class GroupByMonthFilter extends GroupByFilter {
-
-  private int slidingStepsInMo;
-  private int intervalInMo;
   private Calendar calendar = Calendar.getInstance();
-  private static final long MS_TO_MONTH = 30 * 86400_000L;
-  /** 10.31 -> 11.30 -> 12.31, not 10.31 -> 11.30 -> 12.30 */
-  private long initialStartTime;
+
+  private TimeDuration originalSlidingStep;
+  private TimeDuration originalInterval;
 
   // These fields will be serialized to remote nodes, as other fields may be 
updated during process
   private TimeZone timeZone;
-  private boolean isSlidingStepByMonth;
-  private boolean isIntervalByMonth;
-  private long originalSlidingStep;
-  private long originalInterval;
   private long originalStartTime;
   private long originalEndTime;
 
+  private TimeUnit currPrecision;
+
+  private long[] startTimes;
+
   public GroupByMonthFilter() {}
 
   public GroupByMonthFilter(
-      long interval,
-      long slidingStep,
+      TimeDuration interval,
+      TimeDuration slidingStep,
       long startTime,
       long endTime,
-      boolean isSlidingStepByMonth,
-      boolean isIntervalByMonth,
-      TimeZone timeZone) {
-    super(interval, slidingStep, startTime, endTime);
+      TimeZone timeZone,
+      TimeUnit currPrecision) {
+    super(startTime, endTime);
     this.originalInterval = interval;
     this.originalSlidingStep = slidingStep;
+    if (!interval.containsMonth()) {
+      this.interval = interval.nonMonthDuration;
+    }
+    if (!slidingStep.containsMonth()) {
+      this.slidingStep = slidingStep.nonMonthDuration;
+    }
     this.originalStartTime = startTime;
     this.originalEndTime = endTime;
-    initMonthGroupByParameters(isSlidingStepByMonth, isIntervalByMonth, 
timeZone);
+    this.currPrecision = currPrecision;
+    initMonthGroupByParameters(timeZone);
   }
 
   public GroupByMonthFilter(GroupByMonthFilter filter) {
     super(filter.interval, filter.slidingStep, filter.startTime, 
filter.endTime);
-    isIntervalByMonth = filter.isIntervalByMonth;
-    isSlidingStepByMonth = filter.isSlidingStepByMonth;
-    intervalInMo = filter.intervalInMo;
-    slidingStepsInMo = filter.slidingStepsInMo;
-    initialStartTime = filter.initialStartTime;
     originalStartTime = filter.originalStartTime;
     originalEndTime = filter.originalEndTime;
     originalSlidingStep = filter.originalSlidingStep;
@@ -85,17 +89,20 @@ public class GroupByMonthFilter extends GroupByFilter {
     calendar.setTimeZone(filter.calendar.getTimeZone());
     calendar.setTimeInMillis(filter.calendar.getTimeInMillis());
     timeZone = filter.timeZone;
+    currPrecision = filter.currPrecision;
+    // the value in this array will not be changed, so we can copy reference 
directly
+    startTimes = filter.startTimes;
   }
 
   // TODO: time descending order
   @Override
   public boolean satisfy(long time, Object value) {
-    if (time < initialStartTime || time >= endTime) {
+    if (time < originalStartTime || time >= endTime) {
       return false;
     } else if (time >= startTime && time < startTime + slidingStep) {
       return time - startTime < interval;
     } else {
-      long count = getTimePointPosition(time);
+      int count = getTimePointPosition(time);
       getNthTimeInterval(count);
       return time - startTime < interval;
     }
@@ -107,7 +114,7 @@ public class GroupByMonthFilter extends GroupByFilter {
       return true;
     } else {
       // get the interval which contains the start time
-      long count = getTimePointPosition(startTime);
+      int count = getTimePointPosition(startTime);
       getNthTimeInterval(count);
       // judge two adjacent intervals
       if (satisfyCurrentInterval(startTime, endTime)) {
@@ -138,7 +145,7 @@ public class GroupByMonthFilter extends GroupByFilter {
       return true;
     } else {
       // get the interval which contains the start time
-      long count = getTimePointPosition(startTime);
+      int count = getTimePointPosition(startTime);
       getNthTimeInterval(count);
       // judge single interval that contains start time
       return isContainedByCurrentInterval(startTime, endTime);
@@ -149,12 +156,11 @@ public class GroupByMonthFilter extends GroupByFilter {
   public void serialize(DataOutputStream outputStream) {
     try {
       outputStream.write(getSerializeId().ordinal());
-      ReadWriteIOUtils.write(originalInterval, outputStream);
-      ReadWriteIOUtils.write(originalSlidingStep, outputStream);
+      originalInterval.serialize(outputStream);
+      originalSlidingStep.serialize(outputStream);
       ReadWriteIOUtils.write(originalStartTime, outputStream);
       ReadWriteIOUtils.write(originalEndTime, outputStream);
-      ReadWriteIOUtils.write(isSlidingStepByMonth, outputStream);
-      ReadWriteIOUtils.write(isIntervalByMonth, outputStream);
+      ReadWriteIOUtils.write(currPrecision.ordinal(), outputStream);
       ReadWriteIOUtils.write(timeZone.getID(), outputStream);
     } catch (IOException ignored) {
       // ignored
@@ -163,20 +169,21 @@ public class GroupByMonthFilter extends GroupByFilter {
 
   @Override
   public void deserialize(ByteBuffer buffer) {
-    originalInterval = ReadWriteIOUtils.readLong(buffer);
-    originalSlidingStep = ReadWriteIOUtils.readLong(buffer);
+    originalInterval = TimeDuration.deserialize(buffer);
+    originalSlidingStep = TimeDuration.deserialize(buffer);
+    if (!originalInterval.containsMonth()) {
+      this.interval = originalInterval.nonMonthDuration;
+    }
+    if (!originalSlidingStep.containsMonth()) {
+      this.slidingStep = originalSlidingStep.nonMonthDuration;
+    }
     originalStartTime = ReadWriteIOUtils.readLong(buffer);
     originalEndTime = ReadWriteIOUtils.readLong(buffer);
-    isSlidingStepByMonth = ReadWriteIOUtils.readBool(buffer);
-    isIntervalByMonth = ReadWriteIOUtils.readBool(buffer);
+    currPrecision = TimeUnit.values()[ReadWriteIOUtils.readInt(buffer)];
     timeZone = TimeZone.getTimeZone(ReadWriteIOUtils.readString(buffer));
-
-    interval = originalInterval;
-    slidingStep = originalSlidingStep;
     startTime = originalStartTime;
     endTime = originalEndTime;
-
-    initMonthGroupByParameters(isSlidingStepByMonth, isIntervalByMonth, 
timeZone);
+    initMonthGroupByParameters(timeZone);
   }
 
   private boolean isContainedByCurrentInterval(long startTime, long endTime) {
@@ -193,89 +200,67 @@ public class GroupByMonthFilter extends GroupByFilter {
       return false;
     }
     GroupByMonthFilter other = (GroupByMonthFilter) obj;
-    return this.originalInterval == other.originalInterval
-        && this.originalSlidingStep == other.originalSlidingStep
+    return this.originalInterval.equals(other.originalInterval)
+        && this.originalSlidingStep.equals(other.originalSlidingStep)
         && this.originalStartTime == other.originalStartTime
         && this.originalEndTime == other.originalEndTime
-        && this.isSlidingStepByMonth == other.isSlidingStepByMonth
-        && this.isIntervalByMonth == other.isIntervalByMonth
-        && this.timeZone.equals(other.timeZone)
-        && this.initialStartTime == other.initialStartTime
-        && this.intervalInMo == other.intervalInMo
-        && this.slidingStepsInMo == other.slidingStepsInMo;
+        && this.currPrecision == other.currPrecision
+        && this.timeZone.equals(other.timeZone);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(
-        interval, slidingStep, startTime, endTime, isSlidingStepByMonth, 
isIntervalByMonth);
+    return Objects.hash(originalInterval, originalSlidingStep, 
originalStartTime, originalEndTime);
   }
 
-  private void initMonthGroupByParameters(
-      boolean isSlidingStepByMonth, boolean isIntervalByMonth, TimeZone 
timeZone) {
-    initialStartTime = startTime;
+  private void initMonthGroupByParameters(TimeZone timeZone) {
     calendar.setTimeZone(timeZone);
-    calendar.setTimeInMillis(startTime);
     this.timeZone = timeZone;
-    this.isIntervalByMonth = isIntervalByMonth;
-    this.isSlidingStepByMonth = isSlidingStepByMonth;
-    if (isIntervalByMonth) {
-      // TODO: 1mo1d
-      intervalInMo = (int) (interval / MS_TO_MONTH);
-    }
-    if (isSlidingStepByMonth) {
-      slidingStepsInMo = (int) (slidingStep / MS_TO_MONTH);
+    calendar.setTimeInMillis(startTime);
+    if (originalSlidingStep.containsMonth()) {
+      startTimes =
+          getConsecutiveTimesIntervalByMonth(
+              startTime,
+              originalSlidingStep,
+              (int)
+                  Math.ceil(
+                      ((originalEndTime - originalStartTime)
+                          / (double) 
originalSlidingStep.getMinTotalDuration(currPrecision))),
+              timeZone,
+              currPrecision);
     }
     getNthTimeInterval(0);
   }
 
   /** Get the interval that @param time belongs to. */
-  private long getTimePointPosition(long time) {
-    long count;
-    if (isSlidingStepByMonth) {
-      count = (time - this.initialStartTime) / (slidingStepsInMo * 31 * 
86400_000L);
-      calendar.setTimeInMillis(initialStartTime);
-      calendar.add(Calendar.MONTH, (int) count * slidingStepsInMo);
-      while (calendar.getTimeInMillis() < time) {
-        calendar.setTimeInMillis(initialStartTime);
-        calendar.add(Calendar.MONTH, (int) (count + 1) * slidingStepsInMo);
-        if (calendar.getTimeInMillis() > time) {
-          break;
-        } else {
-          count++;
-        }
-      }
+  private int getTimePointPosition(long time) {
+    if (originalSlidingStep.containsMonth()) {
+      int searchResult = Arrays.binarySearch(startTimes, time);
+      return searchResult >= 0 ? searchResult : Math.max(0, 
Math.abs(searchResult) - 2);
     } else {
-      count = (time - this.initialStartTime) / slidingStep;
+      return (int) ((time - originalStartTime) / slidingStep);
     }
-    return count;
   }
 
   /** get the Nth time interval. */
-  private void getNthTimeInterval(long n) {
-    // get start time of time interval
-    if (isSlidingStepByMonth) {
-      calendar.setTimeInMillis(initialStartTime);
-      calendar.add(Calendar.MONTH, (int) (slidingStepsInMo * n));
-    } else {
-      calendar.setTimeInMillis(initialStartTime + slidingStep * n);
-    }
-    this.startTime = calendar.getTimeInMillis();
-
+  private void getNthTimeInterval(int n) {
     // get interval and sliding step
-    if (isIntervalByMonth) {
-      if (isSlidingStepByMonth) {
-        calendar.setTimeInMillis(initialStartTime);
-        calendar.add(Calendar.MONTH, (int) (slidingStepsInMo * n) + 
intervalInMo);
-      } else {
-        calendar.add(Calendar.MONTH, intervalInMo);
+    if (originalSlidingStep.containsMonth()) {
+      if (n < 0 || n > startTimes.length - 1) {
+        this.interval = -1;
+        return;
       }
-      this.interval = calendar.getTimeInMillis() - startTime;
+      this.startTime = startTimes[n];
+      this.slidingStep =
+          calcPositiveIntervalByMonth(startTime, originalSlidingStep, 1, 
timeZone, currPrecision)
+              - startTime;
+    } else {
+      startTime = originalStartTime + n * slidingStep;
     }
-    if (isSlidingStepByMonth) {
-      calendar.setTimeInMillis(initialStartTime);
-      calendar.add(Calendar.MONTH, (int) (slidingStepsInMo * (n + 1)));
-      this.slidingStep = calendar.getTimeInMillis() - startTime;
+    if (originalInterval.containsMonth()) {
+      this.interval =
+          calcPositiveIntervalByMonth(startTime, originalInterval, 1, 
timeZone, currPrecision)
+              - startTime;
     }
   }
 
diff --git 
a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TimeDuration.java
 
b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TimeDuration.java
new file mode 100644
index 00000000000..507a76f00cc
--- /dev/null
+++ 
b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/TimeDuration.java
@@ -0,0 +1,194 @@
+/*
+ * 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.iotdb.tsfile.utils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Calendar;
+import java.util.Objects;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+public class TimeDuration implements Serializable {
+  // month part of time duration
+  public final int monthDuration;
+  // non-month part of time duration, its precision is same as current 
time_precision
+  public final long nonMonthDuration;
+
+  public TimeDuration(int monthDuration, long nonMonthDuration) {
+    this.monthDuration = monthDuration;
+    this.nonMonthDuration = nonMonthDuration;
+  }
+
+  public boolean containsMonth() {
+    return monthDuration != 0;
+  }
+
+  /**
+   * Convert monthDuration to current precision duration, then add 
currPrecisionDuration field.
+   * Think month as 30 days.
+   *
+   * @return the total duration of this timeDuration in current precision
+   */
+  public long getTotalDuration(TimeUnit currPrecision) {
+    return currPrecision.convert(monthDuration * 30 * 86400_000L, 
TimeUnit.MILLISECONDS)
+        + nonMonthDuration;
+  }
+
+  /** Think month as 31 days. */
+  public long getMaxTotalDuration(TimeUnit currPrecision) {
+    return currPrecision.convert(monthDuration * 31 * 86400_000L, 
TimeUnit.MILLISECONDS)
+        + nonMonthDuration;
+  }
+
+  /** Think month as 28 days. */
+  public long getMinTotalDuration(TimeUnit currPrecision) {
+    return currPrecision.convert(monthDuration * 28 * 86400_000L, 
TimeUnit.MILLISECONDS)
+        + nonMonthDuration;
+  }
+
+  public void serialize(ByteBuffer buffer) {
+    ReadWriteIOUtils.write(monthDuration, buffer);
+    ReadWriteIOUtils.write(nonMonthDuration, buffer);
+  }
+
+  public void serialize(DataOutputStream stream) throws IOException {
+    ReadWriteIOUtils.write(monthDuration, stream);
+    ReadWriteIOUtils.write(nonMonthDuration, stream);
+  }
+
+  public static TimeDuration deserialize(ByteBuffer buffer) {
+    return new TimeDuration(ReadWriteIOUtils.readInt(buffer), 
ReadWriteIOUtils.readLong(buffer));
+  }
+
+  /** Get a series of time which duration contains month. */
+  public static long[] getConsecutiveTimesIntervalByMonth(
+      long startTime,
+      TimeDuration duration,
+      int length,
+      TimeZone timeZone,
+      TimeUnit currPrecision) {
+    long[] result = new long[length];
+    result[0] = startTime;
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(timeZone);
+    for (int i = 1; i < length; i++) {
+      result[i] = getStartTime(result[i - 1], duration, currPrecision, 
calendar);
+    }
+    return result;
+  }
+
+  /**
+   * Add several time durations contains natural months based on the startTime 
and avoid edge cases,
+   * ie 2/28
+   *
+   * @param startTime start time
+   * @param duration one duration
+   * @param times num of duration elapsed
+   * @return the time after durations elapsed
+   */
+  public static long calcPositiveIntervalByMonth(
+      long startTime,
+      TimeDuration duration,
+      long times,
+      TimeZone timeZone,
+      TimeUnit currPrecision) {
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(timeZone);
+    for (int i = 0; i < times; i++) {
+      startTime = getStartTime(startTime, duration, currPrecision, calendar);
+    }
+    return startTime;
+  }
+
+  private static long getStartTime(
+      long startTime, TimeDuration duration, TimeUnit currPrecision, Calendar 
calendar) {
+    long coarserThanMsPart = getCoarserThanMsPart(startTime, currPrecision);
+    calendar.setTimeInMillis(coarserThanMsPart);
+    boolean isLastDayOfMonth =
+        calendar.get(Calendar.DAY_OF_MONTH) == 
calendar.getActualMaximum(Calendar.DAY_OF_MONTH);
+    calendar.add(Calendar.MONTH, duration.monthDuration);
+    if (isLastDayOfMonth) {
+      calendar.set(Calendar.DAY_OF_MONTH, 
calendar.getActualMaximum(Calendar.DAY_OF_MONTH));
+    }
+    return currPrecision.convert(calendar.getTimeInMillis(), 
TimeUnit.MILLISECONDS)
+        + getFinerThanMsPart(startTime, currPrecision)
+        + duration.nonMonthDuration;
+  }
+
+  /**
+   * subtract time duration contains natural months based on the startTime
+   *
+   * @param startTime start time
+   * @param duration the duration
+   * @return the time before duration
+   */
+  public static long calcNegativeIntervalByMonth(
+      long startTime, TimeDuration duration, TimeZone timeZone, TimeUnit 
currPrecision) {
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(timeZone);
+    long timeBeforeMonthElapsedInMs =
+        TimeUnit.MILLISECONDS.convert(startTime - duration.nonMonthDuration, 
currPrecision);
+    calendar.setTimeInMillis(timeBeforeMonthElapsedInMs);
+    boolean isLastDayOfMonth =
+        calendar.get(Calendar.DAY_OF_MONTH) == 
calendar.getActualMaximum(Calendar.DAY_OF_MONTH);
+    calendar.add(Calendar.MONTH, -duration.monthDuration);
+    if (isLastDayOfMonth) {
+      calendar.set(Calendar.DAY_OF_MONTH, 
calendar.getActualMaximum(Calendar.DAY_OF_MONTH));
+    }
+    return currPrecision.convert(calendar.getTimeInMillis(), 
TimeUnit.MILLISECONDS)
+        + getFinerThanMsPart(startTime - duration.nonMonthDuration, 
currPrecision);
+  }
+
+  private static long getCoarserThanMsPart(long time, TimeUnit currPrecision) {
+    return TimeUnit.MILLISECONDS.convert(time, currPrecision);
+  }
+
+  private static long getFinerThanMsPart(long time, TimeUnit currPrecision) {
+    switch (currPrecision) {
+      case MILLISECONDS:
+        return 0;
+      case MICROSECONDS:
+        return time % 1000;
+      case NANOSECONDS:
+        return time % 1000_000;
+      default:
+        throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TimeDuration that = (TimeDuration) o;
+    return monthDuration == that.monthDuration && nonMonthDuration == 
that.nonMonthDuration;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(monthDuration, nonMonthDuration);
+  }
+}
diff --git 
a/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/FilterSerializeTest.java
 
b/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/FilterSerializeTest.java
index 2d3b5c0abfa..38bc54af9f0 100644
--- 
a/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/FilterSerializeTest.java
+++ 
b/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/FilterSerializeTest.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.tsfile.read.filter;
 
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.junit.Test;
 
@@ -29,6 +30,7 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 
@@ -109,8 +111,20 @@ public class FilterSerializeTest {
   public void testGroupByMonthFilter() {
     Filter[] filters =
         new Filter[] {
-          new GroupByMonthFilter(1, 2, 3, 4, true, false, 
TimeZone.getTimeZone("Asia/Shanghai")),
-          new GroupByMonthFilter(4, 3, 2, 1, false, true, 
TimeZone.getTimeZone("Atlantic/Faeroe")),
+          new GroupByMonthFilter(
+              new TimeDuration(0, 1),
+              new TimeDuration(0, 2),
+              3,
+              4,
+              TimeZone.getTimeZone("Asia/Shanghai"),
+              TimeUnit.MILLISECONDS),
+          new GroupByMonthFilter(
+              new TimeDuration(0, 4),
+              new TimeDuration(0, 3),
+              2,
+              1,
+              TimeZone.getTimeZone("Atlantic/Faeroe"),
+              TimeUnit.MILLISECONDS),
         };
     for (Filter filter : filters) {
       validateSerialization(filter);
diff --git 
a/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
 
b/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
index a374c7429d7..c22575b4871 100644
--- 
a/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
+++ 
b/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/read/filter/GroupByMonthFilterTest.java
@@ -21,10 +21,12 @@ package org.apache.iotdb.tsfile.read.filter;
 import org.apache.iotdb.tsfile.file.metadata.statistics.LongStatistics;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.utils.TimeDuration;
 
 import org.junit.Test;
 
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -44,7 +46,12 @@ public class GroupByMonthFilterTest {
   public void TestSatisfy1() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_MONTH, 2 * MS_TO_MONTH, 0, END_TIME, true, true, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(1, 0),
+            new TimeDuration(2, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -79,7 +86,12 @@ public class GroupByMonthFilterTest {
   public void TestSatisfy2() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(1, 0),
+            new TimeDuration(1, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -111,7 +123,12 @@ public class GroupByMonthFilterTest {
   public void TestSatisfy3() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(0, MS_TO_DAY),
+            new TimeDuration(1, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -143,7 +160,12 @@ public class GroupByMonthFilterTest {
   public void TestSatisfy4() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_MONTH, MS_TO_DAY * 100, 0, END_TIME, false, true, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(1, 0),
+            new TimeDuration(0, MS_TO_DAY * 100),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
 
     // 1970-01-01 08:00:00, timezone = GMT+08:00
     assertTrue(filter.satisfy(0, null));
@@ -163,7 +185,12 @@ public class GroupByMonthFilterTest {
   public void TestSatisfyStartEndTime() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(0, MS_TO_DAY),
+            new TimeDuration(1, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
 
     // 1970-01-01 08:00:00 - 1970-01-02 08:00:00, timezone = GMT+08:00
     Statistics statistics = new LongStatistics();
@@ -210,7 +237,12 @@ public class GroupByMonthFilterTest {
   public void TestContainStartEndTime() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(0, MS_TO_DAY),
+            new TimeDuration(1, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
 
     // 1970-01-01 08:00:00 - 1970-01-02 08:00:00, timezone = GMT+08:00
     assertFalse(filter.containStartEndTime(0, MS_TO_DAY));
@@ -247,12 +279,24 @@ public class GroupByMonthFilterTest {
   public void TestEquals() {
     GroupByMonthFilter filter =
         new GroupByMonthFilter(
-            MS_TO_DAY, MS_TO_MONTH, 0, END_TIME, true, false, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(0, MS_TO_DAY),
+            new TimeDuration(1, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
+
     Filter filter2 = filter.copy();
     assertEquals(filter, filter2);
     GroupByMonthFilter filter3 =
         new GroupByMonthFilter(
-            MS_TO_MONTH, MS_TO_MONTH, 0, END_TIME, true, true, 
TimeZone.getTimeZone("+08:00"));
+            new TimeDuration(1, 0),
+            new TimeDuration(1, 0),
+            0,
+            END_TIME,
+            TimeZone.getTimeZone("+08:00"),
+            TimeUnit.MILLISECONDS);
+
     assertNotEquals(filter, filter3);
   }
 }
diff --git 
a/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TimeDurationTest.java
 
b/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TimeDurationTest.java
new file mode 100644
index 00000000000..ef3154f9590
--- /dev/null
+++ 
b/iotdb-core/tsfile/src/test/java/org/apache/iotdb/tsfile/utils/TimeDurationTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.iotdb.tsfile.utils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.sql.Timestamp;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.iotdb.tsfile.utils.TimeDuration.calcNegativeIntervalByMonth;
+import static 
org.apache.iotdb.tsfile.utils.TimeDuration.calcPositiveIntervalByMonth;
+
+public class TimeDurationTest {
+  @Test
+  public void calculateIntervalTest() {
+    // 1mo1d duration after 2023-01-31
+    long result =
+        calcPositiveIntervalByMonth(
+            Timestamp.valueOf("2023-01-31 00:00:00").getTime(),
+            new TimeDuration(1, 86400_000),
+            1,
+            TimeZone.getDefault(),
+            TimeUnit.MILLISECONDS);
+    Assert.assertEquals(Timestamp.valueOf("2023-03-01 00:00:00").getTime(), 
result);
+    // 1mo1d duration before 2023-03-01
+    result =
+        calcNegativeIntervalByMonth(
+            result, new TimeDuration(1, 86400_000), TimeZone.getDefault(), 
TimeUnit.MILLISECONDS);
+    Assert.assertEquals(Timestamp.valueOf("2023-01-31 00:00:00").getTime(), 
result);
+
+    // 1mo1d1ns duration after 2023-01-31
+    result =
+        calcPositiveIntervalByMonth(
+            Timestamp.valueOf("2023-01-31 00:00:00").getTime() * 1000_000,
+            new TimeDuration(1, 86400_000_000_001L),
+            1,
+            TimeZone.getDefault(),
+            TimeUnit.NANOSECONDS);
+    Assert.assertEquals(Timestamp.valueOf("2023-03-01 00:00:00").getTime() * 
1000_000 + 1, result);
+    // 1mo1d1ns duration before 2023-03-01
+    result =
+        calcNegativeIntervalByMonth(
+            result,
+            new TimeDuration(1, 86400_000_000_001L),
+            TimeZone.getDefault(),
+            TimeUnit.NANOSECONDS);
+    Assert.assertEquals(Timestamp.valueOf("2023-01-31 00:00:00").getTime() * 
1000_000, result);
+  }
+}

Reply via email to