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/10 10:00:51 UTC

[GitHub] [iotdb] liuminghui233 opened a new pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

liuminghui233 opened a new pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196


   The GROUP BY QUERY statement provides users with three parameters:
   
   - Parameter 1: The display window on the time axis.
   - Parameter 2: Time interval for dividing the time axis (should be positive).
   - Parameter 3: Time sliding step (optional and defaults to equal the time interval if not set).
   
   Now, the time sliding stepn should not be smaller than the time interval. We should support time sliding step less than the time interval.
   
   **Design doc:** https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=199536679
   **JIRA link:** https://issues.apache.org/jira/projects/IOTDB/issues/IOTDB-2590


-- 
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



[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

Posted by GitBox <gi...@apache.org>.
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



[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

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825576326



##########
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:
       I haven't investigated too deeply of the differences between these queues yet. Maybe they can be placed in a separate package and renamed with an aggregate-name-free name first, and each `constructAggrQueue` implementation just codes as ` return new XXXQueue()`. After that, let's see how to refine these queue implementations themselves.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826538127



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/EmptyQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,13 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MinTimeFirstValueDescSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
-  public MinTimeFirstValueDescSlidingWindowAggrQueue(
+public class EmptyQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826083189



##########
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:
       I modified the implementation of the class and now it is the max size.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825563422



##########
File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
##########
@@ -34,12 +34,19 @@
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-public abstract class AggregateResult {
+public abstract class AggregateResult implements Cloneable {
 
   public static final int TIME_LENGTH_FOR_FIRST_VALUE = 100;
   private final AggregationType aggregationType;
   protected TSDataType resultDataType;
 
+  /**
+   * For [COUNT, AVG, SUM], it is the start time of the aggregation window. For [MAX_VALUE,
+   * MIN_VALUE, EXTREME, FIRST_VALUE, LAST_VALUE], it is the timestamp of the current value. For
+   * [MAX_TIME, MIN_TIME], it is always null.
+   */
+  protected long timestamp;

Review comment:
       remember to check whether it's ok to use in case of left open interval.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
##########
@@ -200,28 +161,62 @@ public boolean hasNextWithoutConstraint() {
     return true;
   }
 
-  /**
-   * add natural months based on the startTime to avoid edge cases, ie 2/28
-   *
-   * @param startTime current start time
-   * @param numMonths numMonths is updated in hasNextWithoutConstraint()
-   * @return nextStartTime
-   */
-  public static long calcIntervalByMonth(long startTime, long numMonths) {
-    Calendar calendar = Calendar.getInstance();
-    calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone());
-    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));
+  @Override
+  public RowRecord nextWithoutConstraint() throws IOException {
+    if (!hasCachedTimeInterval) {
+      throw new IOException(
+          "need to call hasNext() before calling next()" + " in GroupByWithValueFilterDataSet.");
     }
-    return calendar.getTimeInMillis();
+    hasCachedTimeInterval = false;
+    curAggregateResults = getNextAggregateResult();
+    return constructRowRecord(curAggregateResults);
   }
 
-  @Override
-  public abstract RowRecord nextWithoutConstraint() throws IOException;
+  protected AggregateResult[] getNextAggregateResult() throws IOException {
+    throw new UnsupportedOperationException("Should call exact sub class!");

Review comment:
       Try to make it an abstract method instead of throwing Exception.

##########
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:
       +1

##########
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 {

Review comment:
       It should not be a `XXXQueue`, you should rename it to `XXXPreExecutor` or something else. And don't forget to add some java doc for this class to explain the functionality of this class and its sub class.

##########
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;
+
+  private TimeSelector timeBoundaryHeap;
+  private long lastTime;

Review comment:
       We don't need this field at all. You can implement a peekFirst() method in `TimeSelector` class and use that here.

##########
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;
+
+  private TimeSelector timeBoundaryHeap;
+  private long lastTime;
+
+  public OverlappedAggrWindowWithNaturalMonthIterator(
+      long startTime,
+      long endTime,
+      long interval,
+      long slidingStep,
+      boolean isAscending,
+      boolean isSlidingStepByMonth,
+      boolean isIntervalByMonth) {
+    this.startTime = startTime;
+    this.endTime = endTime;
+    this.interval = interval;
+    this.slidingStep = slidingStep;
+    this.isAscending = isAscending;
+    this.isSlidingStepByMonth = isSlidingStepByMonth;
+    this.isIntervalByMonth = isIntervalByMonth;
+    initHeap();
+  }
+
+  @Override
+  public Pair<Long, Long> getFirstTimeRange() {
+    long retStartTime = timeBoundaryHeap.pollFirst();
+    lastTime = timeBoundaryHeap.pollFirst();
+    return new Pair<>(retStartTime, lastTime);
+  }
+
+  @Override
+  public Pair<Long, Long> getNextTimeRange(long curStartTime) {
+    if (timeBoundaryHeap.isEmpty()) {
+      return null;
+    }
+    long retStartTime = lastTime;
+    lastTime = timeBoundaryHeap.pollFirst();
+    return new Pair<>(retStartTime, lastTime);
+  }
+
+  private void initHeap() {

Review comment:
       Don't generate all the time interval at the very beginning, try to use the lazy way.

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

Review comment:
       `Extreme`, `Max`, `Min` has the sane logic, only difference is their `judge` method, maybe you should combine them together.

##########
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:
       `Overlapped` is a little bit confusing, maybe rename it to `PreAggrWindowIterator`

##########
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:
       It also think it's better to do so. And in the meanwhile, some Aggregation function share the same logic, if we implement it as an inner class of each aggregation result, we also need to figure out a way to avoid code duplication. Any good ideas about how to avoid these code duplication? @ericpai 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825752862



##########
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:
       fixed




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47368948/badge)](https://coveralls.io/builds/47368948)
   
   Coverage decreased (-2.3%) to 65.6% when pulling **d28ce5ee3558f3acae440f5dd91b2f9be507bc01 on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



[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

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826614680



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(

Review comment:
       > Unfortunately, `TSDataType` is in `tsfile` module and `AggregateResult` is in `server` module, so we cannot add a method in `TSDataType` to return `Comparator<AggregateResult>`.
   > 
   > The best way is to move all these common class into a `common` module, but that has nothing to do with this pr, so you can try another way that is add `private static final Comparator<AggregateResult> comparators[] = new Comparator[6];` in `SlidingWindowGroupByExecutorFactory` and use `comparators[dataType.ordinal()]` to get corresponding comparator.
   
   If so, just use a `Map<TSDataType, Comparator>` instead of indexing by enum's `ordinal()`, which may depend on the definition order.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825656813



##########
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;
+
+  private TimeSelector timeBoundaryHeap;
+  private long lastTime;

Review comment:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1064692057


   Is this ready to review?


-- 
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



[GitHub] [iotdb] JackieTien97 merged pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
JackieTien97 merged pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196


   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825625029



##########
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:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825628911



##########
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:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826120731



##########
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:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826533555



##########
File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/RemovableAggregateResult.java
##########
@@ -17,29 +17,9 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.aggregation;
 
-import org.apache.iotdb.db.query.aggregation.AggregateResult;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+public interface RemovableAggregateResult {

Review comment:
       added




-- 
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



[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

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826635821



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -20,16 +20,51 @@
 package org.apache.iotdb.db.query.executor.groupby;
 
 import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
 import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
 import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
 import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
-import org.apache.iotdb.db.utils.TypeInferenceUtils;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
+import java.util.Comparator;
+
 public class SlidingWindowGroupByExecutorFactory {
 
+  /** comparators used for MonotonicQueueSlidingWindowGroupByExecutor */
+
+  // return a value greater than 0 if o1 is numerically greater than o2
+  private static final Comparator<AggregateResult>[] maxComparators =

Review comment:
       It's better to use `Map<TSDataType, Comparator>` than `ordinal()` indexed because it's an implict restrict that we can't change the order of `TSDataType` definition. If someone changed it,  some bugs will be out.




-- 
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



[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

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826513328



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/SmoothQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,14 +17,17 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.aggregation.RemovableAggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class SumAvgCountSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
+public class SmoothQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       Add comments for this executor to explain its main function.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/EmptyQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,13 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MinTimeFirstValueDescSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
-  public MinTimeFirstValueDescSlidingWindowAggrQueue(
+public class EmptyQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       Add comments for this executor to explain its main function.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/MonotonicQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,26 +17,32 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MaxValueSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
+import java.util.Comparator;
 
-  public MaxValueSlidingWindowAggrQueue(
-      TSDataType dataType, String aggrFuncName, boolean ascending) {
+public class MonotonicQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       Add comments for this executor to explain its main function.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(

Review comment:
       I'm not sure the what the performance impaction is of the boxing/unboxing. Is there any other way to avoid them?
   Maybe we can use primitive type comparsion directly according to the input dataType?

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(
+            dataType, aggrFuncName, ascending, (o1, o2) -> ((Comparable<Object>) o1).compareTo(o2));
+      case SQLConstant.MIN_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(
+            dataType, aggrFuncName, ascending, (o1, o2) -> ((Comparable<Object>) o2).compareTo(o1));
+      case SQLConstant.EXTREME:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(
+            dataType,
+            aggrFuncName,
+            ascending,
+            (o1, o2) -> {
+              TSDataType resultDataType = getResultDataType(dataType, aggrFuncName);
+              Comparable<Object> extVal = (Comparable<Object>) o1;
+              Comparable<Object> absExtVal = (Comparable<Object>) getAbsValue(o1, resultDataType);
+              Comparable<Object> candidateResult = (Comparable<Object>) o2;
+              Comparable<Object> absCandidateResult =
+                  (Comparable<Object>) getAbsValue(o2, resultDataType);
+              if (absExtVal.compareTo(absCandidateResult) == 0) {
+                return extVal.compareTo(candidateResult);
+              } else {
+                return absExtVal.compareTo(absCandidateResult);
+              }
+            });
+      case SQLConstant.MIN_TIME:
+      case SQLConstant.FIRST_VALUE:
+        return !ascending
+            ? new EmptyQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending)
+            : new NormalQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_TIME:
+      case SQLConstant.LAST_VALUE:
+        return !ascending
+            ? new NormalQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending)
+            : new EmptyQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      default:
+        throw new IllegalArgumentException("Invalid Aggregation Type: " + aggrFuncName);
+    }
+  }
+
+  private static Object getAbsValue(Object v, TSDataType resultDataType) {
+    switch (resultDataType) {
+      case DOUBLE:
+        return Math.abs((Double) v);
+      case FLOAT:
+        return Math.abs((Float) v);
+      case INT32:
+        return Math.abs((Integer) v);
+      case INT64:
+        return Math.abs((Long) v);
+      default:
+        throw new UnSupportedDataTypeException(java.lang.String.valueOf(resultDataType));
+    }
+  }
+
+  private static TSDataType getResultDataType(TSDataType dataType, String aggrFuncName) {

Review comment:
       There's a same method in `TypeInferenceUtils.getAggrDataType`. I think we can remove this and leave that.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/RemovableAggregateResult.java
##########
@@ -17,29 +17,9 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.aggregation;
 
-import org.apache.iotdb.db.query.aggregation.AggregateResult;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+public interface RemovableAggregateResult {

Review comment:
       You can add a bit of comments of this interface here.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/NormalQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MinTimeFirstValueSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
+public class NormalQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       Add comments for this executor to explain its main function.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825622236



##########
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:
       fixed




-- 
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



[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

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r824331705



##########
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 `void remove(AggregateResult another)`. 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?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826124402



##########
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:
       added

##########
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:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1067473789


   @ericpai @JackieTien97 PTAL :)


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826535195



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(

Review comment:
       @JackieTien97 What do you think?




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47342665/badge)](https://coveralls.io/builds/47342665)
   
   Coverage decreased (-2.3%) to 65.555% when pulling **5ca6a4951079a242e76179bbf64738eaed4dcea9 on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826544677



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/SmoothQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,14 +17,17 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.aggregation.RemovableAggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class SumAvgCountSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
+public class SmoothQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825640878



##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
##########
@@ -200,28 +161,62 @@ public boolean hasNextWithoutConstraint() {
     return true;
   }
 
-  /**
-   * add natural months based on the startTime to avoid edge cases, ie 2/28
-   *
-   * @param startTime current start time
-   * @param numMonths numMonths is updated in hasNextWithoutConstraint()
-   * @return nextStartTime
-   */
-  public static long calcIntervalByMonth(long startTime, long numMonths) {
-    Calendar calendar = Calendar.getInstance();
-    calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone());
-    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));
+  @Override
+  public RowRecord nextWithoutConstraint() throws IOException {
+    if (!hasCachedTimeInterval) {
+      throw new IOException(
+          "need to call hasNext() before calling next()" + " in GroupByWithValueFilterDataSet.");
     }
-    return calendar.getTimeInMillis();
+    hasCachedTimeInterval = false;
+    curAggregateResults = getNextAggregateResult();
+    return constructRowRecord(curAggregateResults);
   }
 
-  @Override
-  public abstract RowRecord nextWithoutConstraint() throws IOException;
+  protected AggregateResult[] getNextAggregateResult() throws IOException {
+    throw new UnsupportedOperationException("Should call exact sub class!");

Review comment:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826053522



##########
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:
       Because the member variables in the class are all basic data types, a shallow copy is enough.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826105467



##########
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:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826549279



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/MonotonicQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,26 +17,32 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MaxValueSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
+import java.util.Comparator;
 
-  public MaxValueSlidingWindowAggrQueue(
-      TSDataType dataType, String aggrFuncName, boolean ascending) {
+public class MonotonicQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       added

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/impl/NormalQueueSlidingWindowGroupByExecutor.java
##########
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.query.dataset.groupby.queue;
+package org.apache.iotdb.db.query.executor.groupby.impl;
 
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.executor.groupby.SlidingWindowGroupByExecutor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-public class MinTimeFirstValueSlidingWindowAggrQueue extends SlidingWindowAggrQueue {
+public class NormalQueueSlidingWindowGroupByExecutor extends SlidingWindowGroupByExecutor {

Review comment:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47235633/badge)](https://coveralls.io/builds/47235633)
   
   Coverage decreased (-0.01%) to 67.841% when pulling **7f5ccf9f5148cbeeb0fefa147038e328edf1dddd on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47247162/badge)](https://coveralls.io/builds/47247162)
   
   Coverage increased (+0.02%) to 67.87% when pulling **bd5f9ec17f82c6f93d850295992c9ff37cda24eb on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825644086



##########
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 {

Review comment:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826047413



##########
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:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826120881



##########
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:
       added




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826644488



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -20,16 +20,51 @@
 package org.apache.iotdb.db.query.executor.groupby;
 
 import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
 import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
 import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
 import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
-import org.apache.iotdb.db.utils.TypeInferenceUtils;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
+import java.util.Comparator;
+
 public class SlidingWindowGroupByExecutorFactory {
 
+  /** comparators used for MonotonicQueueSlidingWindowGroupByExecutor */
+
+  // return a value greater than 0 if o1 is numerically greater than o2
+  private static final Comparator<AggregateResult>[] maxComparators =

Review comment:
       fixed~

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(

Review comment:
       fixed~




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47366338/badge)](https://coveralls.io/builds/47366338)
   
   Coverage decreased (-2.3%) to 65.563% when pulling **d28ce5ee3558f3acae440f5dd91b2f9be507bc01 on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826047935



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

Review comment:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825739128



##########
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;
+
+  private TimeSelector timeBoundaryHeap;
+  private long lastTime;
+
+  public OverlappedAggrWindowWithNaturalMonthIterator(
+      long startTime,
+      long endTime,
+      long interval,
+      long slidingStep,
+      boolean isAscending,
+      boolean isSlidingStepByMonth,
+      boolean isIntervalByMonth) {
+    this.startTime = startTime;
+    this.endTime = endTime;
+    this.interval = interval;
+    this.slidingStep = slidingStep;
+    this.isAscending = isAscending;
+    this.isSlidingStepByMonth = isSlidingStepByMonth;
+    this.isIntervalByMonth = isIntervalByMonth;
+    initHeap();
+  }
+
+  @Override
+  public Pair<Long, Long> getFirstTimeRange() {
+    long retStartTime = timeBoundaryHeap.pollFirst();
+    lastTime = timeBoundaryHeap.pollFirst();
+    return new Pair<>(retStartTime, lastTime);
+  }
+
+  @Override
+  public Pair<Long, Long> getNextTimeRange(long curStartTime) {
+    if (timeBoundaryHeap.isEmpty()) {
+      return null;
+    }
+    long retStartTime = lastTime;
+    lastTime = timeBoundaryHeap.pollFirst();
+    return new Pair<>(retStartTime, lastTime);
+  }
+
+  private void initHeap() {

Review comment:
       fixed




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47246585/badge)](https://coveralls.io/builds/47246585)
   
   Coverage decreased (-0.008%) to 67.846% when pulling **bd5f9ec17f82c6f93d850295992c9ff37cda24eb on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825622330



##########
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:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826081317



##########
File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
##########
@@ -34,12 +34,19 @@
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-public abstract class AggregateResult {
+public abstract class AggregateResult implements Cloneable {
 
   public static final int TIME_LENGTH_FOR_FIRST_VALUE = 100;
   private final AggregationType aggregationType;
   protected TSDataType resultDataType;
 
+  /**
+   * For [COUNT, AVG, SUM], it is the start time of the aggregation window. For [MAX_VALUE,
+   * MIN_VALUE, EXTREME, FIRST_VALUE, LAST_VALUE], it is the timestamp of the current value. For
+   * [MAX_TIME, MIN_TIME], it is always null.
+   */
+  protected long timestamp;

Review comment:
       I has checked ~ no problem




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47340609/badge)](https://coveralls.io/builds/47340609)
   
   Coverage decreased (-2.3%) to 65.558% when pulling **5ca6a4951079a242e76179bbf64738eaed4dcea9 on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826057205



##########
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:
       I renamed these classes with the algorithm description instead of the aggregate function, maybe this problem doesn't exist now.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825626224



##########
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:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r825751517



##########
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:
       Yeah, because group by natural month doesn't support order by time desc now.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826532269



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(
+            dataType, aggrFuncName, ascending, (o1, o2) -> ((Comparable<Object>) o1).compareTo(o2));
+      case SQLConstant.MIN_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(
+            dataType, aggrFuncName, ascending, (o1, o2) -> ((Comparable<Object>) o2).compareTo(o1));
+      case SQLConstant.EXTREME:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(
+            dataType,
+            aggrFuncName,
+            ascending,
+            (o1, o2) -> {
+              TSDataType resultDataType = getResultDataType(dataType, aggrFuncName);
+              Comparable<Object> extVal = (Comparable<Object>) o1;
+              Comparable<Object> absExtVal = (Comparable<Object>) getAbsValue(o1, resultDataType);
+              Comparable<Object> candidateResult = (Comparable<Object>) o2;
+              Comparable<Object> absCandidateResult =
+                  (Comparable<Object>) getAbsValue(o2, resultDataType);
+              if (absExtVal.compareTo(absCandidateResult) == 0) {
+                return extVal.compareTo(candidateResult);
+              } else {
+                return absExtVal.compareTo(absCandidateResult);
+              }
+            });
+      case SQLConstant.MIN_TIME:
+      case SQLConstant.FIRST_VALUE:
+        return !ascending
+            ? new EmptyQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending)
+            : new NormalQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_TIME:
+      case SQLConstant.LAST_VALUE:
+        return !ascending
+            ? new NormalQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending)
+            : new EmptyQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      default:
+        throw new IllegalArgumentException("Invalid Aggregation Type: " + aggrFuncName);
+    }
+  }
+
+  private static Object getAbsValue(Object v, TSDataType resultDataType) {
+    switch (resultDataType) {
+      case DOUBLE:
+        return Math.abs((Double) v);
+      case FLOAT:
+        return Math.abs((Float) v);
+      case INT32:
+        return Math.abs((Integer) v);
+      case INT64:
+        return Math.abs((Long) v);
+      default:
+        throw new UnSupportedDataTypeException(java.lang.String.valueOf(resultDataType));
+    }
+  }
+
+  private static TSDataType getResultDataType(TSDataType dataType, String aggrFuncName) {

Review comment:
       fixed




-- 
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



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

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826594188



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(

Review comment:
       We can add `createComparator` method in `TSDataType` to save the boxing and unboxing overhead.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#discussion_r826611375



##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/groupby/SlidingWindowGroupByExecutorFactory.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.executor.groupby;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.query.executor.groupby.impl.EmptyQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.MonotonicQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.NormalQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.db.query.executor.groupby.impl.SmoothQueueSlidingWindowGroupByExecutor;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SlidingWindowGroupByExecutorFactory {
+
+  public static SlidingWindowGroupByExecutor getSlidingWindowGroupByExecutor(
+      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 SmoothQueueSlidingWindowGroupByExecutor(dataType, aggrFuncName, ascending);
+      case SQLConstant.MAX_VALUE:
+        return new MonotonicQueueSlidingWindowGroupByExecutor(

Review comment:
       Unfortunately, `TSDataType` is in `tsfile` module and `AggregateResult` is in `server` module, so we cannot add a method in `TSDataType` to return `Comparator<AggregateResult>`. 
   
   The best way is to move all these common class into a `common` module, but that has nothing to do with this pr, so you can try another way that is add `private static final Comparator<AggregateResult> comparators[] = new Comparator[6];` in `SlidingWindowGroupByExecutorFactory` and use `comparators[dataType.ordinal()]` to get corresponding comparator.




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #5196: [IOTDB-2590] Group by time query: support time sliding step less than the time interval

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1063937496


   
   [![Coverage Status](https://coveralls.io/builds/47366535/badge)](https://coveralls.io/builds/47366535)
   
   Coverage decreased (-2.3%) to 65.567% when pulling **d28ce5ee3558f3acae440f5dd91b2f9be507bc01 on liuminghui233:GroupByExtension** into **f133c92e202508e9bec4294cf56f34fc4e67ff9a on apache:master**.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5196:
URL: https://github.com/apache/iotdb/pull/5196#issuecomment-1067649995


   Excellent work!
   LGTM~


-- 
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