You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/03/11 03:58:32 UTC

[GitHub] [iotdb] ericpai commented on a change in pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

ericpai commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r824316079



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/aligned/IoTDBGroupBySlidingWindowQueryWithValueFilterIT.java
##########
@@ -0,0 +1,949 @@
+/*
+ * 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.integration.aligned;
+
+import org.apache.iotdb.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.apache.iotdb.db.constant.TestConstant.avg;
+import static org.apache.iotdb.db.constant.TestConstant.count;
+import static org.apache.iotdb.db.constant.TestConstant.firstValue;
+import static org.apache.iotdb.db.constant.TestConstant.lastValue;
+import static org.apache.iotdb.db.constant.TestConstant.maxTime;
+import static org.apache.iotdb.db.constant.TestConstant.maxValue;
+import static org.apache.iotdb.db.constant.TestConstant.minTime;
+import static org.apache.iotdb.db.constant.TestConstant.minValue;
+import static org.apache.iotdb.db.constant.TestConstant.sum;
+
+@Category({LocalStandaloneTest.class})

Review comment:
       Maybe the category can add `RemoteTest.class` and `ClusterTest.class` thus these iT cases can be verified in a cluster.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/AggrWindowIterator.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.utils.timerangeiterator;
+
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class AggrWindowIterator implements ITimeRangeIterator {
+
+  // total query [startTime, endTime)
+  private final long startTime;
+  private final long endTime;
+
+  private final long interval;
+  private final long slidingStep;
+
+  private final boolean isAscending;
+  private final boolean isSlidingStepByMonth;
+  private final boolean isIntervalByMonth;
+
+  private static final long MS_TO_MONTH = 30 * 86400_000L;

Review comment:
       This constant is decleared in more than 2 classes, maybe we can move it to some class storing constants?

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/OverlappedAggrWindowIterator.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.utils.timerangeiterator;
+
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class OverlappedAggrWindowIterator implements ITimeRangeIterator {

Review comment:
       Please add comments to explain the job of this class.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/queue/SumAvgCountSlidingWindowAggrQueue.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.query.dataset.groupby.queue;
+
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SumAvgCountSlidingWindowAggrQueue extends SlidingWindowAggrQueue {

Review comment:
       As the review comments below, why we should separate the queue as these different implementations?

##########
File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
##########
@@ -113,6 +120,12 @@ public abstract void updateResultUsingValues(
   /** Merge another aggregateResult into this */
   public abstract void merge(AggregateResult another);
 
+  /**
+   * Remove another aggregateResult from this. Note: aggregation window of another must be a subset
+   * of the current aggregation window.
+   */
+  public abstract void remove(AggregateResult another);

Review comment:
       As not all aggregation results can have a remove algorithm. It's not a good design in OOP to add an abstract method which will not be implemented in sub-classes. And in your `ExtremeAggrResult`, it throws an Exception. Throwing exceptions frequently in Java is a huge cost. If it's a non-error flow, it's not suggested to use exception as a flow control.
   IMO, it's better to declare a separate interface called `RemovableAggregateResult`, which contains a method called `AggregateResult`. If one aggregate result supports a 'remove' algorithm, it can implement this interface itself. And for the caller, it just needs to check whether the aggregateResult implements the interface. If so, it can call this method after a simple type convertion. 
   
   What do you think?

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/OverlappedAggrWindowWithNaturalMonthIterator.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.utils.timerangeiterator;
+
+import org.apache.iotdb.db.utils.datastructure.TimeSelector;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class OverlappedAggrWindowWithNaturalMonthIterator implements ITimeRangeIterator {

Review comment:
       Please add comments to explain the job of this class.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/ITimeRangeIterator.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.utils.timerangeiterator;
+
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public interface ITimeRangeIterator {
+
+  Pair<Long, Long> getFirstTimeRange();
+
+  Pair<Long, Long> getNextTimeRange(long curStartTime);
+
+  @TestOnly
+  boolean isAscending();

Review comment:
       I think `isAscending` is a useful method, there's no need to restrict it being used in tests.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/TimeRangeIteratorFactory.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.utils.timerangeiterator;
+
+public class TimeRangeIteratorFactory {

Review comment:
       Please add comments for each new class.
   And maybe we need to add a javadoc styled comments for the method `getTimeRangeIterator` parameters.
   Especially the `interval` and `slidingStep`: they stands for the milliseconds if not grouped by month, or the month count if grouped by month.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/AggrWindowIterator.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.utils.timerangeiterator;
+
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class AggrWindowIterator implements ITimeRangeIterator {

Review comment:
       Please add comments about the job of this class.

##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/aligned/IoTDBGroupBySlidingWindowQueryWithoutValueFilterIT.java
##########
@@ -0,0 +1,938 @@
+/*
+ * 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.integration.aligned;
+
+import org.apache.iotdb.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.apache.iotdb.db.constant.TestConstant.*;
+import static org.apache.iotdb.db.constant.TestConstant.lastValue;
+
+@Category({LocalStandaloneTest.class})

Review comment:
       Maybe the category can add `RemoteTest.class` and `ClusterTest.class` thus these iT cases can be verified in a cluster.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/factory/AggregateResultFactory.java
##########
@@ -134,4 +135,36 @@ public static AggregateResult getAggrResultByType(
         throw new IllegalArgumentException("Invalid Aggregation Type: " + aggregationType.name());
     }
   }
+
+  public static SlidingWindowAggrQueue getSlidingWindowAggrQueueByName(
+      String aggrFuncName, TSDataType dataType, boolean ascending) {
+    if (aggrFuncName == null) {
+      throw new IllegalArgumentException("AggregateFunction Name must not be null");
+    }
+
+    switch (aggrFuncName.toLowerCase()) {
+      case SQLConstant.SUM:
+      case SQLConstant.AVG:
+      case SQLConstant.COUNT:
+        return new SumAvgCountSlidingWindowAggrQueue(dataType, aggrFuncName, ascending);

Review comment:
       These queue names are very strange. If I add another aggreateResult named `X` in the furture, which words the name will be changed to? 
   I think it is better to add an  **abstract** method called 'constructAggrQueue' in `AggregateResult`. Each sub classes will implement it and return a new `AggrQueue`. The aggr queue can code as an innter class of each aggregation result.  Then this method is not needed any more. This is a OOP style design.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/queue/SlidingWindowAggrQueue.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.query.dataset.groupby.queue;
+
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.Deque;
+import java.util.LinkedList;
+
+public abstract class SlidingWindowAggrQueue {
+
+  // current aggregate window
+  protected long curStartTime;
+  protected long curEndTime;
+
+  protected Deque<AggregateResult> deque;
+
+  // output aggregate result
+  protected AggregateResult aggregateResult;
+
+  public SlidingWindowAggrQueue(
+      TSDataType dataType, java.lang.String aggrFuncName, boolean ascending) {

Review comment:
       Use `String` directly.

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/OverlappedAggrWindowIterator.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.utils.timerangeiterator;
+
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class OverlappedAggrWindowIterator implements ITimeRangeIterator {
+
+  // total query [startTime, endTime)
+  private final long startTime;
+  private final long endTime;
+
+  private final long interval;
+  private final long slidingStep;
+
+  private final boolean isAscending;
+
+  private long curInterval;
+  private long curSlidingStep;
+  private boolean isIntervalCyclicChange = false;
+  private int intervalCnt = 0;
+
+  private static final long MS_TO_MONTH = 30 * 86400_000L;
+
+  public OverlappedAggrWindowIterator(
+      long startTime, long endTime, long interval, long slidingStep, boolean isAscending) {
+    this.startTime = startTime;
+    this.endTime = endTime;
+    this.interval = interval;
+    this.slidingStep = slidingStep;
+    this.isAscending = isAscending;
+    initIntervalAndStep();
+  }
+
+  @Override
+  public Pair<Long, Long> getFirstTimeRange() {
+    if (isAscending) {
+      return getLeftmostTimeRange();
+    } else {
+      return getRightmostTimeRange();
+    }
+  }
+
+  private Pair<Long, Long> getLeftmostTimeRange() {
+    long retEndTime = Math.min(startTime + curInterval, endTime);
+    updateIntervalAndStep();
+    return new Pair<>(startTime, retEndTime);
+  }
+
+  private Pair<Long, Long> getRightmostTimeRange() {
+    long retStartTime;
+    long retEndTime;
+    long intervalNum = (long) Math.ceil((endTime - startTime) / (double) slidingStep);
+    retStartTime = slidingStep * (intervalNum - 1) + startTime;
+    if (isIntervalCyclicChange && endTime - retStartTime > interval % slidingStep) {
+      retStartTime += interval % slidingStep;
+      updateIntervalAndStep();
+    }
+    retEndTime = Math.min(retStartTime + curInterval, endTime);
+    updateIntervalAndStep();
+    return new Pair<>(retStartTime, retEndTime);
+  }
+
+  @Override
+  public Pair<Long, Long> getNextTimeRange(long curStartTime) {
+    long retStartTime, retEndTime;
+    if (isAscending) {
+      retStartTime = curStartTime + curSlidingStep;
+      // This is an open interval , [0-100)
+      if (retStartTime >= endTime) {
+        return null;
+      }
+    } else {
+      retStartTime = curStartTime - curSlidingStep;
+      if (retStartTime < startTime) {
+        return null;
+      }
+    }
+    retEndTime = Math.min(retStartTime + curInterval, endTime);
+    updateIntervalAndStep();
+    return new Pair<>(retStartTime, retEndTime);
+  }
+
+  private void initIntervalAndStep() {
+    if (slidingStep >= interval) {
+      curInterval = interval;
+      curSlidingStep = slidingStep;
+    } else if (interval % slidingStep == 0) {
+      curInterval = slidingStep;
+      curSlidingStep = slidingStep;
+    } else {
+      isIntervalCyclicChange = true;
+      curInterval = interval % slidingStep;

Review comment:
       Excellent algorithm!

##########
File path: server/src/test/java/org/apache/iotdb/db/utils/TimeRangeIteratorTest.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.utils;
+
+import org.apache.iotdb.db.utils.timerangeiterator.ITimeRangeIterator;
+import org.apache.iotdb.db.utils.timerangeiterator.TimeRangeIteratorFactory;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TimeRangeIteratorTest {
+
+  @Test
+  public void testNotSplitTimeRange() {
+    String[] res = {
+      "<0,4>", "<3,7>", "<6,10>", "<9,13>", "<12,16>", "<15,19>", "<18,22>", "<21,25>", "<24,28>",
+      "<27,31>", "<30,32>"
+    };
+
+    long startTime = 0, endTime = 32, interval = 4, slidingStep = 3;
+
+    ITimeRangeIterator timeRangeIterator =
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            startTime, endTime, interval, slidingStep, true, false, false, false);
+
+    checkRes(timeRangeIterator, res);
+
+    ITimeRangeIterator descTimeRangeIterator =
+        TimeRangeIteratorFactory.getTimeRangeIterator(
+            startTime, endTime, interval, slidingStep, false, false, false, false);
+
+    checkRes(descTimeRangeIterator, res);
+  }
+
+  @Test
+  public void testSplitTimeRange() {
+    String[] res4_1 = {
+      "<0,1>", "<1,2>", "<2,3>", "<3,4>", "<4,5>", "<5,6>", "<6,7>", "<7,8>", "<8,9>", "<9,10>",
+      "<10,11>", "<11,12>", "<12,13>", "<13,14>", "<14,15>", "<15,16>", "<16,17>", "<17,18>",
+      "<18,19>", "<19,20>", "<20,21>", "<21,22>", "<22,23>", "<23,24>", "<24,25>", "<25,26>",
+      "<26,27>", "<27,28>", "<28,29>", "<29,30>", "<30,31>", "<31,32>"
+    };
+    String[] res4_2 = {
+      "<0,2>", "<2,4>", "<4,6>", "<6,8>", "<8,10>", "<10,12>", "<12,14>", "<14,16>", "<16,18>",
+      "<18,20>", "<20,22>", "<22,24>", "<24,26>", "<26,28>", "<28,30>", "<30,32>"
+    };
+    String[] res4_3 = {
+      "<0,1>", "<1,3>", "<3,4>", "<4,6>", "<6,7>", "<7,9>", "<9,10>", "<10,12>", "<12,13>",
+      "<13,15>", "<15,16>", "<16,18>", "<18,19>", "<19,21>", "<21,22>", "<22,24>", "<24,25>",
+      "<25,27>", "<27,28>", "<28,30>", "<30,31>", "<31,32>"
+    };
+    String[] res4_4 = {
+      "<0,4>", "<4,8>", "<8,12>", "<12,16>", "<16,20>", "<20,24>", "<24,28>", "<28,32>"
+    };
+    String[] res4_5 = {"<0,4>", "<5,9>", "<10,14>", "<15,19>", "<20,24>", "<25,29>", "<30,32>"};
+    String[] res4_6 = {"<0,4>", "<6,10>", "<12,16>", "<18,22>", "<24,28>", "<30,32>"};
+
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 1, true, false, false, true),
+        res4_1);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 2, true, false, false, true),
+        res4_2);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 3, true, false, false, true),
+        res4_3);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 4, true, false, false, true),
+        res4_4);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 5, true, false, false, true),
+        res4_5);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 6, true, false, false, true),
+        res4_6);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 1, false, false, false, true),
+        res4_1);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 2, false, false, false, true),
+        res4_2);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 3, false, false, false, true),
+        res4_3);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 4, false, false, false, true),
+        res4_4);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 5, false, false, false, true),
+        res4_5);
+    checkRes(
+        TimeRangeIteratorFactory.getTimeRangeIterator(0, 32, 4, 6, false, false, false, true),
+        res4_6);
+  }
+
+  @Test
+  public void testNaturalMonthTimeRange() {

Review comment:
       It seems that this test lacks of descending order cases?

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
##########
@@ -176,60 +190,46 @@ public void initGroupBy(QueryContext context, GroupByTimePlan groupByTimePlan)
   }
 
   @Override
-  public RowRecord nextWithoutConstraint() throws IOException {
-    if (!hasCachedTimeInterval) {
-      throw new IOException(
-          "need to call hasNext() before calling next() " + "in GroupByWithoutValueFilterDataSet.");
-    }
-    hasCachedTimeInterval = false;
-    RowRecord record;
-    if (leftCRightO) {
-      record = new RowRecord(curStartTime);
-    } else {
-      record = new RowRecord(curEndTime - 1);
-    }
-
-    curAggregateResults = getNextAggregateResult();
-    for (AggregateResult res : curAggregateResults) {
-      if (res == null) {
-        record.addField(null);
-        continue;
-      }
-      record.addField(res.getResult(), res.getResultDataType());
-    }
-    return record;
-  }
-
-  private AggregateResult[] getNextAggregateResult() throws IOException {
+  protected AggregateResult[] getNextAggregateResult() throws IOException {
     curAggregateResults = new AggregateResult[paths.size()];
+    for (SlidingWindowAggrQueue slidingWindowAggrQueue : slidingWindowAggrQueues) {
+      slidingWindowAggrQueue.setTimeRange(curStartTime, curEndTime);
+    }
     try {
-      // get aggregate results of non-aligned series
-      for (Map.Entry<PartialPath, List<Integer>> entry : pathToAggrIndexesMap.entrySet()) {
-        MeasurementPath path = (MeasurementPath) entry.getKey();
-        List<Integer> indexes = entry.getValue();
-        GroupByExecutor groupByExecutor = pathExecutors.get(path);
-        List<AggregateResult> aggregations = groupByExecutor.calcResult(curStartTime, curEndTime);
-        for (int i = 0; i < aggregations.size(); i++) {
-          int resultIndex = indexes.get(i);
-          curAggregateResults[resultIndex] = aggregations.get(i);
+      while (!isEndCal()) {
+        // get pre-aggregate results of non-aligned series
+        for (Map.Entry<PartialPath, List<Integer>> entry : pathToAggrIndexesMap.entrySet()) {
+          MeasurementPath path = (MeasurementPath) entry.getKey();
+          List<Integer> indexes = entry.getValue();
+          GroupByExecutor groupByExecutor = pathExecutors.get(path);
+          List<AggregateResult> aggregations =
+              groupByExecutor.calcResult(curPreAggrStartTime, curPreAggrEndTime);
+          for (int i = 0; i < aggregations.size(); i++) {
+            int resultIndex = indexes.get(i);
+            slidingWindowAggrQueues[resultIndex].update(aggregations.get(i).clone());
+          }
         }
-      }
-      // get aggregate results of aligned series
-      for (Map.Entry<AlignedPath, List<List<Integer>>> entry :
-          alignedPathToAggrIndexesMap.entrySet()) {
-        AlignedPath path = entry.getKey();
-        List<List<Integer>> indexesList = entry.getValue();
-        AlignedGroupByExecutor groupByExecutor = alignedPathExecutors.get(path);
-        List<List<AggregateResult>> aggregationsList =
-            groupByExecutor.calcAlignedResult(curStartTime, curEndTime);
-        for (int i = 0; i < path.getMeasurementList().size(); i++) {
-          List<AggregateResult> aggregations = aggregationsList.get(i);
-          List<Integer> indexes = indexesList.get(i);
-          for (int j = 0; j < aggregations.size(); j++) {
-            int resultIndex = indexes.get(j);
-            curAggregateResults[resultIndex] = aggregations.get(j);
+        // get pre-aggregate results of aligned series
+        for (Map.Entry<AlignedPath, List<List<Integer>>> entry :
+            alignedPathToAggrIndexesMap.entrySet()) {
+          AlignedPath path = entry.getKey();
+          List<List<Integer>> indexesList = entry.getValue();
+          AlignedGroupByExecutor groupByExecutor = alignedPathExecutors.get(path);
+          List<List<AggregateResult>> aggregationsList =
+              groupByExecutor.calcAlignedResult(curPreAggrStartTime, curPreAggrEndTime);
+          for (int i = 0; i < path.getMeasurementList().size(); i++) {
+            List<AggregateResult> aggregations = aggregationsList.get(i);
+            List<Integer> indexes = indexesList.get(i);
+            for (int j = 0; j < aggregations.size(); j++) {
+              int resultIndex = indexes.get(j);
+              slidingWindowAggrQueues[resultIndex].update(aggregations.get(j).clone());
+            }
           }
         }
+        updatePreAggrInterval();
+      }
+      for (int i = 0; i < curAggregateResults.length; i++) {
+        curAggregateResults[i] = slidingWindowAggrQueues[i].getAggregateResult().clone();

Review comment:
       The `clone()` implemented in `AggregateResult` is just a shallow copy, is it satisfied your requriements?

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/ITimeRangeIterator.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.utils.timerangeiterator;
+
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public interface ITimeRangeIterator {

Review comment:
       This interface may need to add some comments about the methods usage. i.e. does the 'first' and  'last' stand for the real time order or sorting order?

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/timerangeiterator/OverlappedAggrWindowWithNaturalMonthIterator.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.utils.timerangeiterator;
+
+import org.apache.iotdb.db.utils.datastructure.TimeSelector;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class OverlappedAggrWindowWithNaturalMonthIterator implements ITimeRangeIterator {
+
+  // total query [startTime, endTime)
+  private final long startTime;
+  private final long endTime;
+
+  private final long interval;
+  private final long slidingStep;
+
+  private final boolean isAscending;
+  private final boolean isSlidingStepByMonth;
+  private final boolean isIntervalByMonth;
+
+  private static final int HEAP_MAX_SIZE = 100;

Review comment:
       ```suggestion
     private static final int HEAP_INIT_SIZE = 100;
   ```
   As it's the init size of `TimeSelector` actually.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org