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/10/22 07:56:24 UTC

[GitHub] [iotdb] ZhanGHanG9991 opened a new pull request, #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

ZhanGHanG9991 opened a new pull request, #7693:
URL: https://github.com/apache/iotdb/pull/7693

   ## Description
   Support event window aggregation on RawDataAggregationOperator
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error 
       conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, 
       design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design 
   (or naming) decision point and compare the alternatives with the designs that you've implemented 
   (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere 
   (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), 
   link to that discussion from this PR description and explain what have changed in your final design 
   compared to your original proposal or the consensus version in the end of the discussion. 
   If something hasn't changed since the original discussion, you can omit a detailed discussion of 
   those aspects of the design here, perhaps apart from brief mentioning for the sake of readability 
   of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   <hr>
   
   This PR has:
   - [x] been self-reviewed.
       - [ ] concurrent read
       - [ ] concurrent write
       - [ ] concurrent read and write 
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. 
   - [x] added or updated version, __license__, or notice information
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious 
     for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold 
     for code coverage.
   - [ ] added integration tests.
   - [ ] been tested in a test IoTDB cluster.
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items 
   apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items 
   from the checklist above are strictly necessary, but it would be very helpful if you at least 
   self-review the PR. -->
   
   <hr>
   
   ##### Key changed/added classes (or packages if there are too many classes) in this PR
   


-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063982572


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/RawDataAggregationOperator.java:
##########
@@ -46,19 +46,23 @@ public class RawDataAggregationOperator extends SingleInputAggregationOperator {
 
   private final IWindowManager windowManager;
 
+  private boolean isSkipping = false;

Review Comment:
   done.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063983302


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;
+
+  public WindowParameter(TSDataType dataType, int controlColumnIndex, boolean needOutputEndTime) {
+    this.dataType = dataType;
+    this.controlColumnIndex = controlColumnIndex;
+    this.needOutputEndTime = needOutputEndTime;
+  }
+
+  public WindowType getWindowType() {
+    return windowType;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(TSDataType dataType) {
+    this.dataType = dataType;
+  }
+
+  public int getControlColumnIndex() {
+    return controlColumnIndex;
+  }
+
+  public void setControlColumnIndex(int controlColumnIndex) {
+    this.controlColumnIndex = controlColumnIndex;
+  }

Review Comment:
   done,



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063982071


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/TimeWindowManager.java:
##########
@@ -35,10 +42,18 @@ public class TimeWindowManager implements IWindowManager {
 
   private boolean needSkip;
 
-  public TimeWindowManager(ITimeRangeIterator timeRangeIterator) {
+  private TimeWindowParameter timeWindowParameter;

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 merged pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

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


-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1064019210


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindow.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventBooleanWindow extends EventWindow {
+
+  protected boolean eventValue;
+
+  private boolean previousEventValue;
+
+  public EventBooleanWindow(EventWindowParameter eventWindowParameter) {
+    super(eventWindowParameter);
+  }
+
+  @Override
+  public void updatePreviousEventValue() {
+    previousEventValue = eventValue;
+  }
+
+  @Override
+  public void mergeOnePoint(Column[] controlTimeAndValueColumn, int index) {
+    long currentTime = controlTimeAndValueColumn[1].getLong(index);
+    // judge whether we need update startTime
+    if (startTime > currentTime) {
+      startTime = currentTime;
+    }
+    // judge whether we need update endTime
+    if (endTime < currentTime) {
+      endTime = currentTime;
+    }
+    // judge whether we need initialize eventValue
+    if (!initializedEventValue) {
+      startTime = currentTime;
+      endTime = currentTime;
+      eventValue = controlTimeAndValueColumn[0].getBoolean(index);
+      initializedEventValue = true;
+    }
+  }
+
+  public boolean getEventValue() {
+    return eventValue;
+  }
+
+  public void setEventValue(boolean eventValue) {
+    this.eventValue = eventValue;
+  }

Review Comment:
   done.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindow.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventBooleanWindow extends EventWindow {
+
+  protected boolean eventValue;
+
+  private boolean previousEventValue;
+
+  public EventBooleanWindow(EventWindowParameter eventWindowParameter) {
+    super(eventWindowParameter);
+  }
+
+  @Override
+  public void updatePreviousEventValue() {
+    previousEventValue = eventValue;
+  }
+
+  @Override
+  public void mergeOnePoint(Column[] controlTimeAndValueColumn, int index) {
+    long currentTime = controlTimeAndValueColumn[1].getLong(index);
+    // judge whether we need update startTime
+    if (startTime > currentTime) {
+      startTime = currentTime;
+    }
+    // judge whether we need update endTime
+    if (endTime < currentTime) {
+      endTime = currentTime;
+    }
+    // judge whether we need initialize eventValue
+    if (!initializedEventValue) {
+      startTime = currentTime;
+      endTime = currentTime;
+      eventValue = controlTimeAndValueColumn[0].getBoolean(index);
+      initializedEventValue = true;
+    }
+  }
+
+  public boolean getEventValue() {
+    return eventValue;
+  }
+
+  public void setEventValue(boolean eventValue) {
+    this.eventValue = eventValue;
+  }
+
+  public boolean getPreviousEventValue() {
+    return previousEventValue;
+  }
+
+  public void setPreviousEventValue(boolean previousEventValue) {
+    this.previousEventValue = previousEventValue;
+  }

Review Comment:
   done.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1056119136


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindowManager.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventBooleanWindowManager extends EventWindowManager {
+
+  public EventBooleanWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+  }
+
+  @Override
+  public void appendAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // If we have appended aggregation result to resultTsBlockBuilder, we need return directly.
+    if (this.hasAppendedResult) {
+      return;
+    }
+    // Append aggregation results to valueColumnBuilders.
+    ColumnBuilder[] columnBuilders =
+        appendOriginAggregationResult(resultTsBlockBuilder, aggregators);
+    // Judge whether we need output event column.
+    if (windowParameter.isNeedOutputEvent()) {
+      columnBuilders[columnBuilders.length - 1].writeBoolean(

Review Comment:
   write in the last column or the first one?



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1056021652


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EqualEventBooleanWindowManager.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public class EqualEventBooleanWindowManager extends EventBooleanWindowManager {
+
+  public EqualEventBooleanWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+    eventWindow = new EqualEventBooleanWindow(windowParameter);
+  }
+
+  @Override
+  public TsBlock skipPointsOutOfCurWindow(TsBlock inputTsBlock) {
+    if (!needSkip) {
+      return inputTsBlock;
+    }
+
+    if (inputTsBlock == null || inputTsBlock.isEmpty()) {
+      return inputTsBlock;
+    }
+
+    Column controlColumn = inputTsBlock.getColumn(windowParameter.getControlColumnIndex());
+    int i = 0, size = inputTsBlock.getPositionCount();
+    for (; i < size; i++) {
+      if (!controlColumn.isNull(i)
+          && controlColumn.getBoolean(i)
+              != ((EqualEventBooleanWindow) eventWindow).getPreviousEventValue()) {

Review Comment:
   ```suggestion
   Math.abs(controlColumn.getDouble(i)
                         - ((VariationEventDoubleWindow) eventWindow).getPreviousEventValue())
                 > windowParameter.getDelta()) {
   ```
   when the value of delta is zero, equalEvent and variationEvent is the same.



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1062184689


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/VariationEventDoubleWindowManager.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public class VariationEventDoubleWindowManager extends EventDoubleWindowManager {
+
+  public VariationEventDoubleWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+    eventWindow = new VariationEventDoubleWindow(windowParameter);
+  }
+
+  @Override
+  public TsBlock skipPointsOutOfCurWindow(TsBlock inputTsBlock) {
+    if (!needSkip) {
+      return inputTsBlock;
+    }
+
+    if (inputTsBlock == null || inputTsBlock.isEmpty()) {
+      return inputTsBlock;
+    }
+
+    Column controlColumn = inputTsBlock.getColumn(windowParameter.getControlColumnIndex());
+    int i = 0, size = inputTsBlock.getPositionCount();
+    for (; i < size; i++) {
+      if (!controlColumn.isNull(i)
+          && Math.abs(
+                  controlColumn.getDouble(i)
+                      - ((VariationEventDoubleWindow) eventWindow).getPreviousEventValue())
+              > windowParameter.getDelta()) {
+        break;
+      }
+    }

Review Comment:
   I've modified them.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063984239


##########
server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java:
##########
@@ -1405,14 +1407,17 @@ public void rawDataAggregationOperatorTest() throws IllegalPathException {
         AggregationUtil.calculateMaxAggregationResultSize(
             aggregationDescriptors, timeRangeIterator, typeProvider);
 
+    WindowParameter windowParameter = new TimeWindowParameter(TSDataType.INT64, 0, false);

Review Comment:
   done.



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1064271315


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/IWindowManager.java:
##########
@@ -88,4 +88,27 @@ public interface IWindowManager {
    * @return whether there are extra points for the next window
    */
   boolean isTsBlockOutOfBound(TsBlock inputTsBlock);
+
+  default List<TSDataType> getResultDataTypes(List<Aggregator> aggregators) {
+    List<TSDataType> dataTypes = new ArrayList<>();
+    for (Aggregator aggregator : aggregators) {
+      dataTypes.addAll(Arrays.asList(aggregator.getOutputType()));
+    }
+    return dataTypes;
+  }
+
+  TsBlockBuilder createResultTsBlockBuilder(List<Aggregator> aggregators);
+
+  void appendAggregationResult(TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators);
+
+  boolean notInitedLastTimeWindow();

Review Comment:
   Done.



-- 
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 diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
liuminghui233 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1045370912


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowManagerFactory.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.timerangeiterator.ITimeRangeIterator;
+import org.apache.iotdb.db.mpp.execution.operator.exception.UnSupportedEventAggregationMeasureTypeException;
+import org.apache.iotdb.db.mpp.execution.operator.exception.UnSupportedEventAggregationWindowTypeException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+
+public class WindowManagerFactory {
+
+  public IWindowManager genWindowManager(

Review Comment:
   ```suggestion
     public static IWindowManager genWindowManager(
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class WindowParameter {

Review Comment:
   The WindowParameter class can be used as an abstract class, and different types of window parameters extend this class to reduce unnecessary parameters.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/exception/UnSupportedEventAggregationWindowTypeException.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.mpp.execution.operator.exception;
+
+import org.apache.iotdb.db.mpp.execution.operator.window.WindowType;
+
+public class UnSupportedEventAggregationWindowTypeException extends RuntimeException {

Review Comment:
   You can use `IllegalArgumentException` directly, we will check for correct semantics in the Analyzer phase.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventWindowManager.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventWindowManager implements IWindowManager {
+
+  protected boolean initialized;
+
+  protected boolean ascending;
+
+  protected boolean needSkip;
+
+  protected boolean hasAppendedResult;
+
+  protected WindowParameter windowParameter;
+
+  protected EventWindow eventWindow;
+
+  protected EventWindowManager(WindowParameter windowParameter, boolean ascending) {
+    this.windowParameter = windowParameter;
+    this.initialized = false;
+    this.ascending = ascending;
+    // At beginning, we do not need to skip inputTsBlock
+    this.needSkip = false;
+    this.hasAppendedResult = false;
+  }
+
+  @Override
+  public boolean isCurWindowInit() {
+    return this.initialized;
+  }
+
+  @Override
+  public void initCurWindow(TsBlock tsBlock) {
+    this.initialized = true;
+    this.hasAppendedResult = false;
+    this.eventWindow.setInitializedEventValue(false);
+  }
+
+  @Override
+  public boolean hasNext(boolean hasMoreData) {
+    return hasMoreData;
+  }
+
+  @Override
+  public void next() {
+    // When we go into next window, we should pay attention to previous window whether all points
+    // belong to previous window have been consumed. If not, we need skip these points.
+    this.needSkip = true;
+    this.initialized = false;
+    this.eventWindow.updatePreviousEventValue();
+  }
+
+  @Override
+  public IWindow getCurWindow() {
+    return eventWindow;
+  }
+
+  @Override
+  public boolean satisfiedCurWindow(TsBlock inputTsBlock) {
+    return true;
+  }
+
+  @Override
+  public boolean isTsBlockOutOfBound(TsBlock inputTsBlock) {
+    return false;
+  }
+
+  @Override
+  public TsBlockBuilder createResultTsBlockBuilder(List<Aggregator> aggregators) {

Review Comment:
   It seems that the case where we need output start time and end time are not considered here.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/exception/UnSupportedEventAggregationMeasureTypeException.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.mpp.execution.operator.exception;
+
+import org.apache.iotdb.db.mpp.execution.operator.window.CompareType;
+
+public class UnSupportedEventAggregationMeasureTypeException extends RuntimeException {

Review Comment:
   You can use `IllegalArgumentException` directly, we will check for correct semantics in the Analyzer phase.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventLongWindowManager.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventLongWindowManager extends EventWindowManager {
+
+  public EventLongWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+  }
+
+  @Override
+  public void appendAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // If we have appended aggregation result to resultTsBlockBuilder, we need return directly.
+    if (this.hasAppendedResult) {
+      return;
+    }
+    // Append aggregation results to valueColumnBuilders.
+    ColumnBuilder[] columnBuilders =
+        appendOriginAggregationResult(resultTsBlockBuilder, aggregators);
+    // Judge whether we need output event column.
+    if (windowParameter.isNeedOutputEvent()) {

Review Comment:
   It seems that the case where we need output start time and end time are not considered here.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventWindow.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Accumulator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventWindow implements IWindow {
+
+  protected WindowParameter windowParameter;
+
+  protected long startTime;

Review Comment:
   How do we record and output the end time?



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EqualEventBooleanWindow.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public class EqualEventBooleanWindow extends EventBooleanWindow {

Review Comment:
   You can consider using `freemarker` to generate code.
   
   refs:
   https://github.com/apache/iotdb/pull/7880
   https://apache-iotdb.feishu.cn/docx/DORAdXrGeoO2MmxkFvtcIYwCnEh



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1060770627


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowManagerFactory.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.timerangeiterator.ITimeRangeIterator;
+import org.apache.iotdb.db.mpp.execution.operator.exception.UnSupportedEventAggregationMeasureTypeException;
+import org.apache.iotdb.db.mpp.execution.operator.exception.UnSupportedEventAggregationWindowTypeException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+
+public class WindowManagerFactory {
+
+  public IWindowManager genWindowManager(

Review Comment:
   I've modified it.



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1061697065


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EqualEventBooleanWindow.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public class EqualEventBooleanWindow extends EventBooleanWindow {

Review Comment:
   OK. I will use freemarker to generate code in the future.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063983275


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;
+
+  public WindowParameter(TSDataType dataType, int controlColumnIndex, boolean needOutputEndTime) {
+    this.dataType = dataType;
+    this.controlColumnIndex = controlColumnIndex;
+    this.needOutputEndTime = needOutputEndTime;
+  }
+
+  public WindowType getWindowType() {
+    return windowType;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(TSDataType dataType) {
+    this.dataType = dataType;
+  }
+
+  public int getControlColumnIndex() {
+    return controlColumnIndex;
+  }
+
+  public void setControlColumnIndex(int controlColumnIndex) {
+    this.controlColumnIndex = controlColumnIndex;
+  }
+
+  public boolean isNeedOutputEndTime() {
+    return needOutputEndTime;
+  }
+
+  public void setNeedOutputEndTime(boolean needOutputEndTime) {
+    this.needOutputEndTime = needOutputEndTime;
+  }

Review Comment:
   done.



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1060803930


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class WindowParameter {

Review Comment:
   I've made WindowParameter an abstract class, and added TimeWindowParameter and EventWindowParameter.



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1061696598


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/exception/UnSupportedEventAggregationWindowTypeException.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.mpp.execution.operator.exception;
+
+import org.apache.iotdb.db.mpp.execution.operator.window.WindowType;
+
+public class UnSupportedEventAggregationWindowTypeException extends RuntimeException {

Review Comment:
   I've changed them to IllegalArgumentException.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/exception/UnSupportedEventAggregationMeasureTypeException.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.mpp.execution.operator.exception;
+
+import org.apache.iotdb.db.mpp.execution.operator.window.CompareType;
+
+public class UnSupportedEventAggregationMeasureTypeException extends RuntimeException {

Review Comment:
   I've changed them to IllegalArgumentException.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063999925


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/TimeWindowManager.java:
##########
@@ -65,14 +80,11 @@ public void next() {
     // belong to previous window have been consumed. If not, we need skip these points.
     this.needSkip = true;
     this.initialized = false;
+    this.startTime = this.curWindow.getCurMinTime();
+    this.endTime = this.curWindow.getCurMaxTime();

Review Comment:
   done. 
   



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1055666156


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventWindowManager.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventWindowManager implements IWindowManager {
+
+  protected boolean initialized;
+
+  protected boolean ascending;
+
+  protected boolean needSkip;
+
+  protected boolean hasAppendedResult;
+
+  protected WindowParameter windowParameter;
+
+  protected EventWindow eventWindow;
+
+  protected EventWindowManager(WindowParameter windowParameter, boolean ascending) {
+    this.windowParameter = windowParameter;
+    this.initialized = false;
+    this.ascending = ascending;
+    // At beginning, we do not need to skip inputTsBlock
+    this.needSkip = false;
+    this.hasAppendedResult = false;
+  }
+
+  @Override
+  public boolean isCurWindowInit() {
+    return this.initialized;
+  }
+
+  @Override
+  public void initCurWindow(TsBlock tsBlock) {
+    this.initialized = true;
+    this.hasAppendedResult = false;
+    this.eventWindow.setInitializedEventValue(false);
+  }
+
+  @Override
+  public boolean hasNext(boolean hasMoreData) {
+    return hasMoreData;
+  }
+
+  @Override
+  public void next() {
+    // When we go into next window, we should pay attention to previous window whether all points
+    // belong to previous window have been consumed. If not, we need skip these points.
+    this.needSkip = true;
+    this.initialized = false;
+    this.eventWindow.updatePreviousEventValue();
+  }
+
+  @Override
+  public IWindow getCurWindow() {
+    return eventWindow;
+  }
+
+  @Override
+  public boolean satisfiedCurWindow(TsBlock inputTsBlock) {
+    return true;
+  }
+
+  @Override
+  public boolean isTsBlockOutOfBound(TsBlock inputTsBlock) {
+    return false;
+  }
+
+  @Override
+  public TsBlockBuilder createResultTsBlockBuilder(List<Aggregator> aggregators) {
+    List<TSDataType> dataTypes = getResultDataTypes(aggregators);
+    // Judge whether we need output event column.
+    if (windowParameter.isNeedOutputEvent()) {
+      dataTypes.add(windowParameter.getDataType());
+    }
+    return new TsBlockBuilder(dataTypes);
+  }
+
+  protected ColumnBuilder[] appendOriginAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // Use the start time of eventWindow as default output time.
+    TimeColumnBuilder timeColumnBuilder = resultTsBlockBuilder.getTimeColumnBuilder();
+    timeColumnBuilder.writeLong(eventWindow.getStartTime());
+
+    ColumnBuilder[] columnBuilders = resultTsBlockBuilder.getValueColumnBuilders();
+    int columnIndex = 0;
+    for (Aggregator aggregator : aggregators) {
+      ColumnBuilder[] columnBuilder = new ColumnBuilder[aggregator.getOutputType().length];
+      columnBuilder[0] = columnBuilders[columnIndex++];
+      if (columnBuilder.length > 1) {
+        columnBuilder[1] = columnBuilders[columnIndex++];
+      }
+      aggregator.outputResult(columnBuilder);
+    }
+    resultTsBlockBuilder.declarePosition();
+    this.hasAppendedResult = true;
+    return columnBuilders;
+  }
+
+  @Override
+  public boolean notInitedLastTimeWindow() {

Review Comment:
   notInitializedLastTimeWindow



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1056021652


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EqualEventBooleanWindowManager.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public class EqualEventBooleanWindowManager extends EventBooleanWindowManager {
+
+  public EqualEventBooleanWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+    eventWindow = new EqualEventBooleanWindow(windowParameter);
+  }
+
+  @Override
+  public TsBlock skipPointsOutOfCurWindow(TsBlock inputTsBlock) {
+    if (!needSkip) {
+      return inputTsBlock;
+    }
+
+    if (inputTsBlock == null || inputTsBlock.isEmpty()) {
+      return inputTsBlock;
+    }
+
+    Column controlColumn = inputTsBlock.getColumn(windowParameter.getControlColumnIndex());
+    int i = 0, size = inputTsBlock.getPositionCount();
+    for (; i < size; i++) {
+      if (!controlColumn.isNull(i)
+          && controlColumn.getBoolean(i)
+              != ((EqualEventBooleanWindow) eventWindow).getPreviousEventValue()) {

Review Comment:
   ```suggestion
   Math.abs(controlColumn.getDouble(i)
                         - ((VariationEventDoubleWindow) eventWindow).getPreviousEventValue())
                 > windowParameter.getDelta()) {
   ```
   when the value of delta is zero, equalEvent and variationEvent is the same.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindowManager.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventBooleanWindowManager extends EventWindowManager {
+
+  public EventBooleanWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+  }
+
+  @Override
+  public void appendAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // If we have appended aggregation result to resultTsBlockBuilder, we need return directly.
+    if (this.hasAppendedResult) {
+      return;
+    }
+    // Append aggregation results to valueColumnBuilders.
+    ColumnBuilder[] columnBuilders =
+        appendOriginAggregationResult(resultTsBlockBuilder, aggregators);
+    // Judge whether we need output event column.
+    if (windowParameter.isNeedOutputEvent()) {
+      columnBuilders[columnBuilders.length - 1].writeBoolean(

Review Comment:
   write in the last column or the first one?



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindowManager.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventBooleanWindowManager extends EventWindowManager {
+
+  public EventBooleanWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+  }
+
+  @Override
+  public void appendAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // If we have appended aggregation result to resultTsBlockBuilder, we need return directly.
+    if (this.hasAppendedResult) {

Review Comment:
   why hasAppendedResult?



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/CompareType.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+public enum CompareType {
+  EQUAL((byte) 0),
+
+  VARIATION((byte) 1);
+
+  private final byte type;
+
+  CompareType(byte type) {
+    this.type = type;
+  }
+
+  public byte getType() {
+    return type;
+  }
+}

Review Comment:
   ```suggestion
   ```
   The design now doesn't distinguish EQUAL and VARIATION. EQUAL is just the VARIATION with zero delta.
   The SQL gramma now is: 
   ```
   select first_value(s1), avg(s1), count(s2), sum(s3) from root.sg.d group by vartiation(s1)  // the equal situation
   
   select avg(s1), count(s2), sum(s3) from root.sg.d group by vartiation(s1,2.2)
   ```
   Can be seen in https://apache-iotdb.feishu.cn/docx/doxcn8DbAHGeQUuwtOaOgsRIBlc



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/VariationEventDoubleWindowManager.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public class VariationEventDoubleWindowManager extends EventDoubleWindowManager {
+
+  public VariationEventDoubleWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+    eventWindow = new VariationEventDoubleWindow(windowParameter);
+  }
+
+  @Override
+  public TsBlock skipPointsOutOfCurWindow(TsBlock inputTsBlock) {
+    if (!needSkip) {
+      return inputTsBlock;
+    }
+
+    if (inputTsBlock == null || inputTsBlock.isEmpty()) {
+      return inputTsBlock;
+    }
+
+    Column controlColumn = inputTsBlock.getColumn(windowParameter.getControlColumnIndex());
+    int i = 0, size = inputTsBlock.getPositionCount();
+    for (; i < size; i++) {
+      if (!controlColumn.isNull(i)
+          && Math.abs(
+                  controlColumn.getDouble(i)
+                      - ((VariationEventDoubleWindow) eventWindow).getPreviousEventValue())
+              > windowParameter.getDelta()) {
+        break;
+      }
+    }

Review Comment:
   ```suggestion
       int i = 0, size = inputTsBlock.getPositionCount();
       double previousEventValue = ((VariationEventDoubleWindow) eventWindow).getPreviousEventValue();
       for (; i < size; i++) {
         if (!controlColumn.isNull(i) && Math.abs(controlColumn.getDouble(i) - previousEventValue > windowParameter.getDelta()) {
           break;
         }
       }
   ```



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1060804526


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventWindow.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Accumulator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventWindow implements IWindow {
+
+  protected WindowParameter windowParameter;
+
+  protected long startTime;

Review Comment:
   We've resolved this issue.



-- 
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 diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063961361


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/SlidingWindowAggregationOperator.java:
##########
@@ -24,8 +24,12 @@
 import org.apache.iotdb.db.mpp.execution.operator.Operator;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
 
+import java.util.ArrayList;
+import java.util.Arrays;

Review Comment:
   SlidingWindowAggregationOperator may also need to output endTime because it's just a speficial case of group by time



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/TimeWindowManager.java:
##########
@@ -35,10 +42,18 @@ public class TimeWindowManager implements IWindowManager {
 
   private boolean needSkip;
 
-  public TimeWindowManager(ITimeRangeIterator timeRangeIterator) {
+  private TimeWindowParameter timeWindowParameter;

Review Comment:
   There is no need to save the whole `TimeWindowParameter`, it seems that you only use one of its field: `needOutputEndTime`, you can  save that boolean in this class.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;

Review Comment:
   ```suggestion
     private final TSDataType dataType;
   
     private final int controlColumnIndex;
   
     private final boolean needOutputEndTime;
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;

Review Comment:
   these two shouldn't put in base class, because TimeWindowParameter don't need these.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/RawDataAggregationOperator.java:
##########
@@ -46,19 +46,23 @@ public class RawDataAggregationOperator extends SingleInputAggregationOperator {
 
   private final IWindowManager windowManager;
 
+  private boolean isSkipping = false;

Review Comment:
   BTW, I think `needSkip` may be easier to understand.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindow.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventBooleanWindow extends EventWindow {
+
+  protected boolean eventValue;
+
+  private boolean previousEventValue;
+
+  public EventBooleanWindow(EventWindowParameter eventWindowParameter) {
+    super(eventWindowParameter);
+  }
+
+  @Override
+  public void updatePreviousEventValue() {
+    previousEventValue = eventValue;
+  }
+
+  @Override
+  public void mergeOnePoint(Column[] controlTimeAndValueColumn, int index) {
+    long currentTime = controlTimeAndValueColumn[1].getLong(index);
+    // judge whether we need update startTime
+    if (startTime > currentTime) {
+      startTime = currentTime;
+    }
+    // judge whether we need update endTime
+    if (endTime < currentTime) {
+      endTime = currentTime;
+    }
+    // judge whether we need initialize eventValue
+    if (!initializedEventValue) {
+      startTime = currentTime;
+      endTime = currentTime;
+      eventValue = controlTimeAndValueColumn[0].getBoolean(index);
+      initializedEventValue = true;
+    }
+  }
+
+  public boolean getEventValue() {
+    return eventValue;
+  }
+
+  public void setEventValue(boolean eventValue) {
+    this.eventValue = eventValue;
+  }
+
+  public boolean getPreviousEventValue() {
+    return previousEventValue;
+  }
+
+  public void setPreviousEventValue(boolean previousEventValue) {
+    this.previousEventValue = previousEventValue;
+  }

Review Comment:
   remember to check other types, I won't comment each one



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindow.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventBooleanWindow extends EventWindow {
+
+  protected boolean eventValue;
+
+  private boolean previousEventValue;
+
+  public EventBooleanWindow(EventWindowParameter eventWindowParameter) {
+    super(eventWindowParameter);
+  }
+
+  @Override
+  public void updatePreviousEventValue() {
+    previousEventValue = eventValue;
+  }
+
+  @Override
+  public void mergeOnePoint(Column[] controlTimeAndValueColumn, int index) {
+    long currentTime = controlTimeAndValueColumn[1].getLong(index);
+    // judge whether we need update startTime
+    if (startTime > currentTime) {
+      startTime = currentTime;
+    }
+    // judge whether we need update endTime
+    if (endTime < currentTime) {
+      endTime = currentTime;
+    }
+    // judge whether we need initialize eventValue
+    if (!initializedEventValue) {
+      startTime = currentTime;
+      endTime = currentTime;
+      eventValue = controlTimeAndValueColumn[0].getBoolean(index);
+      initializedEventValue = true;
+    }
+  }
+
+  public boolean getEventValue() {
+    return eventValue;
+  }
+
+  public void setEventValue(boolean eventValue) {
+    this.eventValue = eventValue;
+  }

Review Comment:
   ```suggestion
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindow.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.read.common.block.column.Column;
+
+public abstract class EventBooleanWindow extends EventWindow {
+
+  protected boolean eventValue;
+
+  private boolean previousEventValue;
+
+  public EventBooleanWindow(EventWindowParameter eventWindowParameter) {
+    super(eventWindowParameter);
+  }
+
+  @Override
+  public void updatePreviousEventValue() {
+    previousEventValue = eventValue;
+  }
+
+  @Override
+  public void mergeOnePoint(Column[] controlTimeAndValueColumn, int index) {
+    long currentTime = controlTimeAndValueColumn[1].getLong(index);
+    // judge whether we need update startTime
+    if (startTime > currentTime) {
+      startTime = currentTime;
+    }
+    // judge whether we need update endTime
+    if (endTime < currentTime) {
+      endTime = currentTime;
+    }
+    // judge whether we need initialize eventValue
+    if (!initializedEventValue) {
+      startTime = currentTime;
+      endTime = currentTime;
+      eventValue = controlTimeAndValueColumn[0].getBoolean(index);
+      initializedEventValue = true;
+    }
+  }
+
+  public boolean getEventValue() {
+    return eventValue;
+  }
+
+  public void setEventValue(boolean eventValue) {
+    this.eventValue = eventValue;
+  }
+
+  public boolean getPreviousEventValue() {
+    return previousEventValue;
+  }
+
+  public void setPreviousEventValue(boolean previousEventValue) {
+    this.previousEventValue = previousEventValue;
+  }

Review Comment:
   ```suggestion
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/TimeWindowManager.java:
##########
@@ -65,14 +80,11 @@ public void next() {
     // belong to previous window have been consumed. If not, we need skip these points.
     this.needSkip = true;
     this.initialized = false;
+    this.startTime = this.curWindow.getCurMinTime();
+    this.endTime = this.curWindow.getCurMaxTime();

Review Comment:
   It depends on `leftCRightO`.
   
   ```
   leftCRightO ? curTimeRange.getMin() : curTimeRange.getMax()
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;
+
+  public WindowParameter(TSDataType dataType, int controlColumnIndex, boolean needOutputEndTime) {
+    this.dataType = dataType;
+    this.controlColumnIndex = controlColumnIndex;
+    this.needOutputEndTime = needOutputEndTime;
+  }
+
+  public WindowType getWindowType() {
+    return windowType;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(TSDataType dataType) {
+    this.dataType = dataType;
+  }
+
+  public int getControlColumnIndex() {
+    return controlColumnIndex;
+  }
+
+  public void setControlColumnIndex(int controlColumnIndex) {
+    this.controlColumnIndex = controlColumnIndex;
+  }

Review Comment:
   ```suggestion
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/RawDataAggregationOperator.java:
##########
@@ -46,19 +46,23 @@ public class RawDataAggregationOperator extends SingleInputAggregationOperator {
 
   private final IWindowManager windowManager;
 
+  private boolean isSkipping = false;

Review Comment:
   add some comments about this important field to describe its meaing.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;
+
+  public WindowParameter(TSDataType dataType, int controlColumnIndex, boolean needOutputEndTime) {
+    this.dataType = dataType;
+    this.controlColumnIndex = controlColumnIndex;
+    this.needOutputEndTime = needOutputEndTime;
+  }
+
+  public WindowType getWindowType() {
+    return windowType;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(TSDataType dataType) {
+    this.dataType = dataType;
+  }

Review Comment:
   ```suggestion
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/IWindowManager.java:
##########
@@ -88,4 +88,27 @@ public interface IWindowManager {
    * @return whether there are extra points for the next window
    */
   boolean isTsBlockOutOfBound(TsBlock inputTsBlock);
+
+  default List<TSDataType> getResultDataTypes(List<Aggregator> aggregators) {
+    List<TSDataType> dataTypes = new ArrayList<>();
+    for (Aggregator aggregator : aggregators) {
+      dataTypes.addAll(Arrays.asList(aggregator.getOutputType()));
+    }
+    return dataTypes;
+  }
+
+  TsBlockBuilder createResultTsBlockBuilder(List<Aggregator> aggregators);
+
+  void appendAggregationResult(TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators);
+
+  boolean notInitedLastTimeWindow();

Review Comment:
   add java doc for these two methods to describe their meanings.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/TimeWindowManager.java:
##########
@@ -65,14 +80,11 @@ public void next() {
     // belong to previous window have been consumed. If not, we need skip these points.
     this.needSkip = true;
     this.initialized = false;
+    this.startTime = this.curWindow.getCurMinTime();
+    this.endTime = this.curWindow.getCurMaxTime();

Review Comment:
   There should be some ITs about these, check that, if it's absent, add it in this pr.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;
+
+  public WindowParameter(TSDataType dataType, int controlColumnIndex, boolean needOutputEndTime) {
+    this.dataType = dataType;
+    this.controlColumnIndex = controlColumnIndex;
+    this.needOutputEndTime = needOutputEndTime;
+  }
+
+  public WindowType getWindowType() {
+    return windowType;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(TSDataType dataType) {
+    this.dataType = dataType;
+  }
+
+  public int getControlColumnIndex() {
+    return controlColumnIndex;
+  }
+
+  public void setControlColumnIndex(int controlColumnIndex) {
+    this.controlColumnIndex = controlColumnIndex;
+  }
+
+  public boolean isNeedOutputEndTime() {
+    return needOutputEndTime;
+  }
+
+  public void setNeedOutputEndTime(boolean needOutputEndTime) {
+    this.needOutputEndTime = needOutputEndTime;
+  }

Review Comment:
   ```suggestion
   ```



##########
server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java:
##########
@@ -1405,14 +1407,17 @@ public void rawDataAggregationOperatorTest() throws IllegalPathException {
         AggregationUtil.calculateMaxAggregationResultSize(
             aggregationDescriptors, timeRangeIterator, typeProvider);
 
+    WindowParameter windowParameter = new TimeWindowParameter(TSDataType.INT64, 0, false);

Review Comment:
   ```suggestion
       WindowParameter windowParameter = new TimeWindowParameter(false);
   ```



##########
server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java:
##########
@@ -1391,13 +1393,16 @@ public Operator visitAggregation(AggregationNode node, LocalExecutionPlanContext
           calculateMaxAggregationResultSize(
               aggregationDescriptors, timeRangeIterator, context.getTypeProvider());
 
+      WindowParameter windowParameter = new TimeWindowParameter(TSDataType.INT64, 0, false);

Review Comment:
   ```suggestion
         WindowParameter windowParameter = new TimeWindowParameter(false);
   ```



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1061679338


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventWindowManager.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.TimeColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventWindowManager implements IWindowManager {
+
+  protected boolean initialized;
+
+  protected boolean ascending;
+
+  protected boolean needSkip;
+
+  protected boolean hasAppendedResult;
+
+  protected WindowParameter windowParameter;
+
+  protected EventWindow eventWindow;
+
+  protected EventWindowManager(WindowParameter windowParameter, boolean ascending) {
+    this.windowParameter = windowParameter;
+    this.initialized = false;
+    this.ascending = ascending;
+    // At beginning, we do not need to skip inputTsBlock
+    this.needSkip = false;
+    this.hasAppendedResult = false;
+  }
+
+  @Override
+  public boolean isCurWindowInit() {
+    return this.initialized;
+  }
+
+  @Override
+  public void initCurWindow(TsBlock tsBlock) {
+    this.initialized = true;
+    this.hasAppendedResult = false;
+    this.eventWindow.setInitializedEventValue(false);
+  }
+
+  @Override
+  public boolean hasNext(boolean hasMoreData) {
+    return hasMoreData;
+  }
+
+  @Override
+  public void next() {
+    // When we go into next window, we should pay attention to previous window whether all points
+    // belong to previous window have been consumed. If not, we need skip these points.
+    this.needSkip = true;
+    this.initialized = false;
+    this.eventWindow.updatePreviousEventValue();
+  }
+
+  @Override
+  public IWindow getCurWindow() {
+    return eventWindow;
+  }
+
+  @Override
+  public boolean satisfiedCurWindow(TsBlock inputTsBlock) {
+    return true;
+  }
+
+  @Override
+  public boolean isTsBlockOutOfBound(TsBlock inputTsBlock) {
+    return false;
+  }
+
+  @Override
+  public TsBlockBuilder createResultTsBlockBuilder(List<Aggregator> aggregators) {

Review Comment:
   I've added it.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventLongWindowManager.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventLongWindowManager extends EventWindowManager {
+
+  public EventLongWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+  }
+
+  @Override
+  public void appendAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // If we have appended aggregation result to resultTsBlockBuilder, we need return directly.
+    if (this.hasAppendedResult) {
+      return;
+    }
+    // Append aggregation results to valueColumnBuilders.
+    ColumnBuilder[] columnBuilders =
+        appendOriginAggregationResult(resultTsBlockBuilder, aggregators);
+    // Judge whether we need output event column.
+    if (windowParameter.isNeedOutputEvent()) {

Review Comment:
   I've added them.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063984206


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;

Review Comment:
   done.



##########
server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/OperatorTreeGenerator.java:
##########
@@ -1391,13 +1393,16 @@ public Operator visitAggregation(AggregationNode node, LocalExecutionPlanContext
           calculateMaxAggregationResultSize(
               aggregationDescriptors, timeRangeIterator, context.getTypeProvider());
 
+      WindowParameter windowParameter = new TimeWindowParameter(TSDataType.INT64, 0, false);

Review Comment:
   done



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1063983335


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;
+
+  public WindowParameter(TSDataType dataType, int controlColumnIndex, boolean needOutputEndTime) {
+    this.dataType = dataType;
+    this.controlColumnIndex = controlColumnIndex;
+    this.needOutputEndTime = needOutputEndTime;
+  }
+
+  public WindowType getWindowType() {
+    return windowType;
+  }
+
+  public TSDataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(TSDataType dataType) {
+    this.dataType = dataType;
+  }

Review Comment:
   done.



##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/WindowParameter.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.execution.operator.window;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class WindowParameter {
+
+  protected WindowType windowType;
+
+  private TSDataType dataType;
+
+  private int controlColumnIndex;
+
+  private boolean needOutputEndTime;

Review Comment:
   done.



-- 
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] ycycse commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ycycse commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1064107773


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/SlidingWindowAggregationOperator.java:
##########
@@ -24,8 +24,12 @@
 import org.apache.iotdb.db.mpp.execution.operator.Operator;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
 
+import java.util.ArrayList;
+import java.util.Arrays;

Review Comment:
   SlidingWindow share a different framework with RawDataAggregator.
   So I think realize it in another PR is better.



-- 
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] ZhanGHanG9991 commented on a diff in pull request #7693: [IOTDB-4437] Implement event window in RawDataAggregationOperator

Posted by GitBox <gi...@apache.org>.
ZhanGHanG9991 commented on code in PR #7693:
URL: https://github.com/apache/iotdb/pull/7693#discussion_r1062173241


##########
server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/window/EventBooleanWindowManager.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.mpp.execution.operator.window;
+
+import org.apache.iotdb.db.mpp.aggregation.Aggregator;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilder;
+
+import java.util.List;
+
+public abstract class EventBooleanWindowManager extends EventWindowManager {
+
+  public EventBooleanWindowManager(WindowParameter windowParameter, boolean ascending) {
+    super(windowParameter, ascending);
+  }
+
+  @Override
+  public void appendAggregationResult(
+      TsBlockBuilder resultTsBlockBuilder, List<Aggregator> aggregators) {
+    // If we have appended aggregation result to resultTsBlockBuilder, we need return directly.
+    if (this.hasAppendedResult) {

Review Comment:
   I've deleted this.



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