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 2020/05/19 02:49:57 UTC

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #1204: [IOTDB-622] add count records for all databases

qiaojialin commented on a change in pull request #1204:
URL: https://github.com/apache/incubator-iotdb/pull/1204#discussion_r426991499



##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
##########
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.dataset.groupby;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COLUMN;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COUNT;
+
+public class GroupByLevelDataSet extends QueryDataSet {
+
+  private static final Logger logger = LoggerFactory
+    .getLogger(GroupByLevelDataSet.class);
+
+  private List<RowRecord> records = new ArrayList<>();
+  private int index = 0;
+
+  private int level = -1;
+  private boolean isByTime = false;
+  protected long queryId;
+  private GroupByEngineDataSet dataSet;
+  private GroupByPlan groupByPlan;
+  private QueryContext context;
+
+  private Map<Path, GroupByExecutor> pathExecutors = new HashMap<>();
+  private Map<Path, List<Integer>> resultIndexes = new HashMap<>();
+
+  public GroupByLevelDataSet(QueryContext context, GroupByPlan plan, GroupByEngineDataSet dataSet)
+    throws QueryProcessException, StorageEngineException, IOException {
+    this.queryId = context.getQueryId();
+    this.level = plan.getLevel();
+    this.isByTime = plan.isByTime();
+    this.dataSet = dataSet;
+    this.paths = plan.getPaths();
+    this.dataTypes = plan.getDataTypes();
+    this.groupByPlan = plan;
+    this.context = context;
+
+    logger.debug("paths " + this.paths);
+
+    RowRecord record = new RowRecord(0);
+    if (!isByTime) {
+      initGroupBy();
+      record = getRecordWithoutTimeInterval();
+    } else {
+      // get all records from GroupByDataSet, then we merge them to one record
+      logger.debug("only group by level, paths:" + groupByPlan.getPaths());
+      while (dataSet != null && dataSet.hasNextWithoutConstraint()) {
+        RowRecord curRecord = dataSet.nextWithoutConstraint();
+        record = mergeRecords(curRecord, record);
+      }
+    }
+
+    // prepare final records
+    // group by level
+    // path -> count
+    Map<String, Long> finalPaths = new TreeMap<String, Long>();
+
+    for (int j = 0; j < this.paths.size(); j++) {
+      String[] tmpPath = this.paths.get(j).getFullPath().split("\\.");
+
+      String key;
+      if (tmpPath.length <= level) {
+        key = this.paths.get(j).getFullPath();
+      } else {
+        StringBuilder path = new StringBuilder();
+        for (int k = 0; k <= level; k++) {
+          if (k == 0) {
+            path.append(tmpPath[k]);
+          } else {
+            path.append("." + tmpPath[k]);
+          }
+        }
+        key = path.toString();
+      }
+      logger.debug("records " + record.getFields().toString());

Review comment:
       add if (logger.isDebugEnabled())

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/crud/GroupByPlan.java
##########
@@ -33,6 +33,8 @@
   // if it is left close and right open interval
   private boolean leftCRightO = true;
 
+  private boolean byTime = false;

Review comment:
       The GroupByPlan is actually GroupByTimePlan, so no need to add a byTime field, we can rename the class.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
##########
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.dataset.groupby;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COLUMN;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COUNT;
+
+public class GroupByLevelDataSet extends QueryDataSet {
+
+  private static final Logger logger = LoggerFactory
+    .getLogger(GroupByLevelDataSet.class);
+
+  private List<RowRecord> records = new ArrayList<>();
+  private int index = 0;
+
+  private int level = -1;
+  private boolean isByTime = false;
+  protected long queryId;
+  private GroupByEngineDataSet dataSet;
+  private GroupByPlan groupByPlan;
+  private QueryContext context;
+
+  private Map<Path, GroupByExecutor> pathExecutors = new HashMap<>();
+  private Map<Path, List<Integer>> resultIndexes = new HashMap<>();
+
+  public GroupByLevelDataSet(QueryContext context, GroupByPlan plan, GroupByEngineDataSet dataSet)
+    throws QueryProcessException, StorageEngineException, IOException {
+    this.queryId = context.getQueryId();
+    this.level = plan.getLevel();
+    this.isByTime = plan.isByTime();
+    this.dataSet = dataSet;
+    this.paths = plan.getPaths();
+    this.dataTypes = plan.getDataTypes();
+    this.groupByPlan = plan;
+    this.context = context;
+
+    logger.debug("paths " + this.paths);
+
+    RowRecord record = new RowRecord(0);
+    if (!isByTime) {
+      initGroupBy();
+      record = getRecordWithoutTimeInterval();
+    } else {
+      // get all records from GroupByDataSet, then we merge them to one record
+      logger.debug("only group by level, paths:" + groupByPlan.getPaths());
+      while (dataSet != null && dataSet.hasNextWithoutConstraint()) {
+        RowRecord curRecord = dataSet.nextWithoutConstraint();
+        record = mergeRecords(curRecord, record);
+      }
+    }
+
+    // prepare final records
+    // group by level
+    // path -> count
+    Map<String, Long> finalPaths = new TreeMap<String, Long>();
+
+    for (int j = 0; j < this.paths.size(); j++) {
+      String[] tmpPath = this.paths.get(j).getFullPath().split("\\.");

Review comment:
       Actually, we support path like this:  root.sg.d."s.1"
   s.1 is measurement name

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
##########
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.dataset.groupby;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COLUMN;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COUNT;
+
+public class GroupByLevelDataSet extends QueryDataSet {
+
+  private static final Logger logger = LoggerFactory
+    .getLogger(GroupByLevelDataSet.class);
+
+  private List<RowRecord> records = new ArrayList<>();
+  private int index = 0;
+
+  private int level = -1;
+  private boolean isByTime = false;
+  protected long queryId;
+  private GroupByEngineDataSet dataSet;
+  private GroupByPlan groupByPlan;
+  private QueryContext context;
+
+  private Map<Path, GroupByExecutor> pathExecutors = new HashMap<>();
+  private Map<Path, List<Integer>> resultIndexes = new HashMap<>();
+
+  public GroupByLevelDataSet(QueryContext context, GroupByPlan plan, GroupByEngineDataSet dataSet)
+    throws QueryProcessException, StorageEngineException, IOException {
+    this.queryId = context.getQueryId();
+    this.level = plan.getLevel();
+    this.isByTime = plan.isByTime();
+    this.dataSet = dataSet;
+    this.paths = plan.getPaths();
+    this.dataTypes = plan.getDataTypes();
+    this.groupByPlan = plan;
+    this.context = context;
+
+    logger.debug("paths " + this.paths);
+
+    RowRecord record = new RowRecord(0);
+    if (!isByTime) {
+      initGroupBy();
+      record = getRecordWithoutTimeInterval();
+    } else {
+      // get all records from GroupByDataSet, then we merge them to one record
+      logger.debug("only group by level, paths:" + groupByPlan.getPaths());
+      while (dataSet != null && dataSet.hasNextWithoutConstraint()) {
+        RowRecord curRecord = dataSet.nextWithoutConstraint();
+        record = mergeRecords(curRecord, record);
+      }
+    }
+
+    // prepare final records
+    // group by level
+    // path -> count
+    Map<String, Long> finalPaths = new TreeMap<String, Long>();
+
+    for (int j = 0; j < this.paths.size(); j++) {
+      String[] tmpPath = this.paths.get(j).getFullPath().split("\\.");
+
+      String key;
+      if (tmpPath.length <= level) {
+        key = this.paths.get(j).getFullPath();
+      } else {
+        StringBuilder path = new StringBuilder();
+        for (int k = 0; k <= level; k++) {
+          if (k == 0) {
+            path.append(tmpPath[k]);
+          } else {
+            path.append("." + tmpPath[k]);
+          }
+        }
+        key = path.toString();
+      }
+      logger.debug("records " + record.getFields().toString());
+      finalPaths.put(key,
+        record.getFields().get(j).getLongV()
+          + finalPaths.getOrDefault(key, 0L));
+    }
+
+    this.paths = Arrays.asList(new Path(COLUMN_COLUMN), new Path(COLUMN_COUNT));
+    this.dataTypes = Arrays.asList(TSDataType.TEXT, TSDataType.TEXT);
+
+    for (Map.Entry<String, Long> entry : finalPaths.entrySet()) {
+      RowRecord tmpRecord = new RowRecord(0);
+      Field field = new Field(TSDataType.TEXT);
+      field.setBinaryV(new Binary(entry.getKey()));
+      logger.debug("rowRecord " + entry.getKey());
+      Field field1 = new Field(TSDataType.TEXT);
+      field1.setBinaryV(new Binary(Long.toString(entry.getValue())));
+      tmpRecord.addField(field);
+      tmpRecord.addField(field1);
+      records.add(tmpRecord);
+      logger.info(tmpRecord.toString());
+    }
+  }
+
+  @Override
+  protected boolean hasNextWithoutConstraint() throws IOException {
+    return index < records.size();
+  }
+
+  @Override
+  protected RowRecord nextWithoutConstraint() {
+    return records.get(index++);
+  }
+
+  private void initGroupBy()
+    throws QueryProcessException, StorageEngineException {
+    // get all aggregation results, then we package them to one record
+    for (int i = 0; i < paths.size(); i++) {
+      Path path = paths.get(i);
+      if (!pathExecutors.containsKey(path)) {
+        //init GroupByExecutor
+        pathExecutors.put(path,
+          getGroupByExecutor(path, groupByPlan.getAllMeasurementsInDevice(path.getDevice()), dataTypes.get(i), this.context, null, null));
+        resultIndexes.put(path, new ArrayList<>());
+      } else {
+        throw new QueryProcessException("duplicated path found, path:" + path);
+      }
+      resultIndexes.get(path).add(i);
+      AggregateResult aggrResult = AggregateResultFactory
+        .getAggrResultByName(groupByPlan.getDeduplicatedAggregations().get(i), dataTypes.get(i));
+      pathExecutors.get(path).addAggregateResult(aggrResult);
+    }
+  }
+
+  private GroupByExecutor getGroupByExecutor(Path path, Set<String> allSensors, TSDataType dataType,
+                                             QueryContext context, Filter timeFilter, TsFileFilter fileFilter)
+    throws StorageEngineException, QueryProcessException {
+    return new LocalGroupByExecutor(path, allSensors, dataType, context, timeFilter, fileFilter);
+  }
+
+  private RowRecord getRecordWithoutTimeInterval()
+    throws IOException {
+    RowRecord record = new RowRecord(0);
+    AggregateResult[] fields = new AggregateResult[paths.size()];
+
+    try {
+      for (Map.Entry<Path, GroupByExecutor> pathToExecutorEntry : pathExecutors.entrySet()) {
+        GroupByExecutor executor = pathToExecutorEntry.getValue();
+        List<AggregateResult> aggregations = executor.calcResult(Long.MIN_VALUE, Long.MAX_VALUE);
+        for (int i = 0; i < aggregations.size(); i++) {
+          int resultIndex = resultIndexes.get(pathToExecutorEntry.getKey()).get(i);
+          fields[resultIndex] = aggregations.get(i);
+        }
+      }
+    } catch (QueryProcessException e) {
+      logger.error("GroupByWithoutValueFilterDataSet execute has error", e);
+      throw new IOException(e.getMessage(), e);
+    }
+
+    for (AggregateResult res : fields) {
+      if (res == null) {
+        record.addField(null);
+        continue;
+      }
+      record.addField(res.getResult(), res.getResultDataType());
+    }
+    return record;
+  }
+
+  private RowRecord mergeRecords(RowRecord newRecord, RowRecord oldRecord) {

Review comment:
       please add javadoc and an example

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
##########
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.dataset.groupby;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COLUMN;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COUNT;
+
+public class GroupByLevelDataSet extends QueryDataSet {
+
+  private static final Logger logger = LoggerFactory
+    .getLogger(GroupByLevelDataSet.class);
+
+  private List<RowRecord> records = new ArrayList<>();
+  private int index = 0;
+
+  private int level = -1;
+  private boolean isByTime = false;
+  protected long queryId;
+  private GroupByEngineDataSet dataSet;
+  private GroupByPlan groupByPlan;
+  private QueryContext context;
+
+  private Map<Path, GroupByExecutor> pathExecutors = new HashMap<>();
+  private Map<Path, List<Integer>> resultIndexes = new HashMap<>();
+
+  public GroupByLevelDataSet(QueryContext context, GroupByPlan plan, GroupByEngineDataSet dataSet)
+    throws QueryProcessException, StorageEngineException, IOException {
+    this.queryId = context.getQueryId();
+    this.level = plan.getLevel();
+    this.isByTime = plan.isByTime();
+    this.dataSet = dataSet;
+    this.paths = plan.getPaths();
+    this.dataTypes = plan.getDataTypes();
+    this.groupByPlan = plan;
+    this.context = context;
+
+    logger.debug("paths " + this.paths);
+
+    RowRecord record = new RowRecord(0);
+    if (!isByTime) {
+      initGroupBy();
+      record = getRecordWithoutTimeInterval();
+    } else {
+      // get all records from GroupByDataSet, then we merge them to one record
+      logger.debug("only group by level, paths:" + groupByPlan.getPaths());
+      while (dataSet != null && dataSet.hasNextWithoutConstraint()) {
+        RowRecord curRecord = dataSet.nextWithoutConstraint();
+        record = mergeRecords(curRecord, record);
+      }
+    }
+
+    // prepare final records
+    // group by level
+    // path -> count
+    Map<String, Long> finalPaths = new TreeMap<String, Long>();
+
+    for (int j = 0; j < this.paths.size(); j++) {
+      String[] tmpPath = this.paths.get(j).getFullPath().split("\\.");
+
+      String key;
+      if (tmpPath.length <= level) {
+        key = this.paths.get(j).getFullPath();
+      } else {
+        StringBuilder path = new StringBuilder();
+        for (int k = 0; k <= level; k++) {
+          if (k == 0) {
+            path.append(tmpPath[k]);
+          } else {
+            path.append("." + tmpPath[k]);
+          }
+        }
+        key = path.toString();
+      }
+      logger.debug("records " + record.getFields().toString());
+      finalPaths.put(key,
+        record.getFields().get(j).getLongV()
+          + finalPaths.getOrDefault(key, 0L));
+    }
+
+    this.paths = Arrays.asList(new Path(COLUMN_COLUMN), new Path(COLUMN_COUNT));
+    this.dataTypes = Arrays.asList(TSDataType.TEXT, TSDataType.TEXT);
+
+    for (Map.Entry<String, Long> entry : finalPaths.entrySet()) {
+      RowRecord tmpRecord = new RowRecord(0);
+      Field field = new Field(TSDataType.TEXT);
+      field.setBinaryV(new Binary(entry.getKey()));
+      logger.debug("rowRecord " + entry.getKey());
+      Field field1 = new Field(TSDataType.TEXT);
+      field1.setBinaryV(new Binary(Long.toString(entry.getValue())));
+      tmpRecord.addField(field);
+      tmpRecord.addField(field1);
+      records.add(tmpRecord);
+      logger.info(tmpRecord.toString());
+    }
+  }
+
+  @Override
+  protected boolean hasNextWithoutConstraint() throws IOException {
+    return index < records.size();
+  }
+
+  @Override
+  protected RowRecord nextWithoutConstraint() {
+    return records.get(index++);
+  }
+
+  private void initGroupBy()

Review comment:
       initGroupByLevel? 

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
##########
@@ -113,34 +114,45 @@ protected AggregationExecutor getAggregationExecutor(AggregationPlan aggregation
 
   @Override
   public QueryDataSet groupBy(GroupByPlan groupByPlan, QueryContext context)
-      throws QueryFilterOptimizationException, StorageEngineException, QueryProcessException {
-    long unit = groupByPlan.getInterval();
-    long slidingStep = groupByPlan.getSlidingStep();
-    long startTime = groupByPlan.getStartTime();
-    long endTime = groupByPlan.getEndTime();
+    throws QueryFilterOptimizationException, StorageEngineException, QueryProcessException, IOException {
+
+    logger.debug("paths:" + groupByPlan.getPaths() + " level:" + groupByPlan.getLevel() + " byTime:" + groupByPlan.isByTime());

Review comment:
       check if debug enabled

##########
File path: server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
##########
@@ -241,6 +241,13 @@ groupByClause
       COMMA DURATION
       (COMMA DURATION)?
       RR_BRACKET
+    | GROUP BY LEVEL OPERATOR_EQ INT
+    | GROUP BY LR_BRACKET
+            timeInterval
+            COMMA DURATION
+            (COMMA DURATION)?
+            RR_BRACKET
+            COMMA LEVEL OPERATOR_EQ INT

Review comment:
       No need to modify the group by ([1,10), 1ms) clause.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByLevelDataSet.java
##########
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.dataset.groupby;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.query.aggregation.AggregateResult;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COLUMN;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_COUNT;
+
+public class GroupByLevelDataSet extends QueryDataSet {
+
+  private static final Logger logger = LoggerFactory
+    .getLogger(GroupByLevelDataSet.class);
+
+  private List<RowRecord> records = new ArrayList<>();
+  private int index = 0;
+
+  private int level = -1;
+  private boolean isByTime = false;
+  protected long queryId;
+  private GroupByEngineDataSet dataSet;
+  private GroupByPlan groupByPlan;

Review comment:
       Replace this with an AggregationPlan

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
##########
@@ -664,6 +668,7 @@ public void enterGroupByFillClause(SqlBaseParser.GroupByFillClauseContext ctx) {
     super.enterGroupByFillClause(ctx);
     queryOp.setGroupBy(true);
     queryOp.setFill(true);
+    queryOp.setGroupByTime(true);

Review comment:
       since there is more than one group by, it's better to use full name.   GroupByLevel and GroupByTime




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

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