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 2021/02/21 23:33:07 UTC

[GitHub] [iotdb] wshao08 opened a new pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

wshao08 opened a new pull request #2714:
URL: https://github.com/apache/iotdb/pull/2714


   ## Description
   Currently, only one aggregated measurement is supported in this statement to avoid result ambiguity. 
   This PR is to support multiple aggregated measurements in group by level statement.
   
   JIRA issue: https://issues.apache.org/jira/browse/IOTDB-1024
   


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



[GitHub] [iotdb] HTHou commented on pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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


   Don't forget to run Run `mvn spotless:apply`.


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



[GitHub] [iotdb] Alima777 commented on a change in pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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



##########
File path: server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationByLevelIT.java
##########
@@ -59,6 +59,10 @@
       };
 
   private static final double DOUBLE_PRECISION = 0.001d;
+  private final String sg1d1 = "root.sg1.d1";
+  private final String sg1d2 = "root.sg1.d2";
+  private final String sg2d1 = "root.sg2.d1";
+  private final String sg2d2 = "root.sg2.d2";

Review comment:
       Remove this which is never used.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/crud/AggregationPlan.java
##########
@@ -67,4 +77,25 @@ public int getLevel() {
   public void setLevel(int level) {
     this.level = level;
   }
+
+  public Map<String, AggregateResult> getPathByLevel() throws QueryProcessException {
+    if (!finalAggreagtionPaths.isEmpty()) {
+      return finalAggreagtionPaths;
+    }
+    List<PartialPath> seriesPaths = getPaths();
+    List<TSDataType> dataTypes = getDataTypes();
+    try {
+      for (int i = 0; i < seriesPaths.size(); i++) {
+        String transformedPath =
+            FilePathUtils.generatePartialPathByLevel(seriesPaths.get(i).getFullPath(), getLevel());
+        String key = getAggregations().get(i) + "(" + transformedPath + ")";
+        AggregateResult aggRet =
+            AggregateResultFactory.getAggrResultByName(getAggregations().get(i), dataTypes.get(i));
+        finalAggreagtionPaths.putIfAbsent(key, aggRet);

Review comment:
       Generate the AggrResult every time is unnecessary.
   ```suggestion
           if (finalAggreagtionPaths.containsKey(key)) {
           AggregateResult aggRet =
               AggregateResultFactory.getAggrResultByName(getAggregations().get(i), dataTypes.get(i));
           finalAggreagtionPaths.put(key, aggRet);
           }   
   ```

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationByLevelIT.java
##########
@@ -191,26 +204,35 @@ public void timeFuncGroupByLevelTest() throws Exception {
   public void valueFuncGroupByLevelTest() throws Exception {
     String[] retArray =
         new String[] {
-          "61.22", "71.12,62.15",
+          "61.22,125.5", "71.12,62.15,71.12,62.15",
         };
     try (Connection connection =
             DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      statement.execute("select last_value(temperature) from root.*.* GROUP BY level=0");
+      statement.execute(
+          "select last_value(temperature),max_value(temperature) from root.*.* GROUP BY level=0");
 
       int cnt = 0;
       try (ResultSet resultSet = statement.getResultSet()) {
         while (resultSet.next()) {
-          String ans = resultSet.getString(1);
+          String ans = resultSet.getString(1) + "," + resultSet.getString(2);
           Assert.assertEquals(retArray[cnt], ans);
           cnt++;
         }
       }
 
-      statement.execute("select max_value(temperature) from root.sg1.* GROUP BY level=2");
+      statement.execute(
+          "select last_value(temperature),max_value(temperature) from root.sg1.* GROUP BY level=2");

Review comment:
       ```suggestion
             "select last_value(temperature), max_value(temperature) from root.sg1.* GROUP BY level=2");
   ```

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationByLevelIT.java
##########
@@ -159,26 +163,35 @@ public void avgFuncGroupByLevelTest() throws Exception {
   public void timeFuncGroupByLevelTest() throws Exception {
     String[] retArray =
         new String[] {
-          "100", "600,700",
+          "8,100", "600,700,2,3",
         };
     try (Connection connection =
             DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      statement.execute("select min_time(temperature) from root.*.* GROUP BY level=0");
+      statement.execute(
+          "select count(status), min_time(temperature) from root.*.* GROUP BY level=0");
 
       int cnt = 0;
       try (ResultSet resultSet = statement.getResultSet()) {
         while (resultSet.next()) {
-          String ans = resultSet.getString(1);
+          String ans = resultSet.getString(1) + "," + resultSet.getString(2);
           Assert.assertEquals(retArray[cnt], ans);
           cnt++;
         }
       }

Review comment:
       Check the metadata result(column name) concurrently because you modified this. And try to check it using different value of level in test.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
##########
@@ -876,6 +877,16 @@ private void deduplicate(QueryPlan queryPlan, int fetchSize)
                   .getContext()
                   .getColumnName()
               : columnForReader;
+      if (queryPlan instanceof AggregationPlan && ((AggregationPlan) queryPlan).getLevel() >= 0) {
+        String aggregatePath =
+            originalPath.isMeasurementAliasExists()
+                ? FilePathUtils.generatePartialPathByLevel(
+                    originalPath.getFullPathWithAlias(), ((AggregationPlan) queryPlan).getLevel())
+                : FilePathUtils.generatePartialPathByLevel(
+                    originalPath.toString(), ((AggregationPlan) queryPlan).getLevel());

Review comment:
       In line 850, check the level and don't change the value of `columnForReader`. In this way, you can use the `columnForReader` directly here.

##########
File path: server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationByLevelIT.java
##########
@@ -191,26 +204,35 @@ public void timeFuncGroupByLevelTest() throws Exception {
   public void valueFuncGroupByLevelTest() throws Exception {
     String[] retArray =
         new String[] {
-          "61.22", "71.12,62.15",
+          "61.22,125.5", "71.12,62.15,71.12,62.15",
         };
     try (Connection connection =
             DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
         Statement statement = connection.createStatement()) {
-      statement.execute("select last_value(temperature) from root.*.* GROUP BY level=0");
+      statement.execute(
+          "select last_value(temperature),max_value(temperature) from root.*.* GROUP BY level=0");

Review comment:
       ```suggestion
             "select last_value(temperature), max_value(temperature) from root.*.* GROUP BY level=0");
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
##########
@@ -195,43 +171,49 @@ public static String getTsFilePrefixPath(TsFileResource resource) {
       if (newRecord.getFields().get(i) == null) {
         aggregateResultList.add(
             AggregateResultFactory.getAggrResultByName(
-                plan.getAggregations().get(i), plan.getDeduplicatedDataTypes().get(i)));
+                plan.getDeduplicatedAggregations().get(i), plan.getDeduplicatedDataTypes().get(i)));
       } else {
         TSDataType dataType = newRecord.getFields().get(i).getDataType();
         AggregateResult aggRet =
-            AggregateResultFactory.getAggrResultByName(plan.getAggregations().get(i), dataType);
-        switch (dataType) {
-          case TEXT:
-            aggRet.setBinaryValue(newRecord.getFields().get(i).getBinaryV());
-            break;
-          case INT32:
-            aggRet.setIntValue(newRecord.getFields().get(i).getIntV());
-            break;
-          case INT64:
-            aggRet.setLongValue(newRecord.getFields().get(i).getLongV());
-            break;
-          case FLOAT:
-            aggRet.setFloatValue(newRecord.getFields().get(i).getFloatV());
-            break;
-          case DOUBLE:
-            aggRet.setDoubleValue(newRecord.getFields().get(i).getDoubleV());
-            break;
-          case BOOLEAN:
-            aggRet.setBooleanValue(newRecord.getFields().get(i).getBoolV());
-            break;
-          default:
-            throw new UnSupportedDataTypeException(dataType.toString());
+            AggregateResultFactory.getAggrResultByName(
+                plan.getDeduplicatedAggregations().get(i), dataType);
+        if (aggRet.getAggregationType().equals(AggregationType.AVG)) {
+          ((AvgAggrResult) aggRet).updateAvg(dataType, newRecord.getFields().get(i).getDoubleV());
+        } else {

Review comment:
       If it's just for here, I think it's better to write a new public method `setAvg`. which set `avg` and `cnt` directly rather than modify the access restrictions of `private` method.




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



[GitHub] [iotdb] HTHou commented on pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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


   By the way, the order of the results is not same with the sql statement.  Is it normal?


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



[GitHub] [iotdb] HTHou removed a comment on pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

Posted by GitBox <gi...@apache.org>.
HTHou removed a comment on pull request #2714:
URL: https://github.com/apache/iotdb/pull/2714#issuecomment-783006971


   By the way, the order of the results is not same with the sql statement.  Is it normal?


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



[GitHub] [iotdb] HTHou edited a comment on pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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


   The order is correct now, but the result of avg seems not correct, which shouldn't be null. 
   ```
   IoTDB> select sum(QPS),avg(QPS), sum(errorQps) from root.yuerj."30001"."client-521".*.* GROUP BY ([1970-01-01T08:00:00.000+08:00, 1970-01-01T08:00:00.900+08:00), 100ms) ,level=3
   +-----------------------------+--------------------------------------------+--------------------------------------------+-------------------------------------------------+
   |                         Time|sum(root.yuerj."30001"."client-521".*.*.QPS)|avg(root.yuerj."30001"."client-521".*.*.QPS)|sum(root.yuerj."30001"."client-521".*.*.errorQps)|
   +-----------------------------+--------------------------------------------+--------------------------------------------+-------------------------------------------------+
   |1970-01-01T08:00:00.000+08:00|                                        15.0|                                        null|                                             10.0|
   |1970-01-01T08:00:00.100+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.200+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.300+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.400+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.500+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.600+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.700+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.800+08:00|                                         0.0|                                        null|                                              0.0|
   +-----------------------------+--------------------------------------------+--------------------------------------------+-------------------------------------------------+
   Total line number = 9
   It costs 0.058s
   ```


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



[GitHub] [iotdb] HTHou merged pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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


   


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



[GitHub] [iotdb] HTHou commented on pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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


   The order is correct now. 
   ```
   IoTDB> select sum(QPS),avg(QPS), sum(errorQps) from root.yuerj."30001"."client-521".*.* GROUP BY ([1970-01-01T08:00:00.000+08:00, 1970-01-01T08:00:00.900+08:00), 100ms) ,level=3
   +-----------------------------+--------------------------------------------+--------------------------------------------+-------------------------------------------------+
   |                         Time|sum(root.yuerj."30001"."client-521".*.*.QPS)|avg(root.yuerj."30001"."client-521".*.*.QPS)|sum(root.yuerj."30001"."client-521".*.*.errorQps)|
   +-----------------------------+--------------------------------------------+--------------------------------------------+-------------------------------------------------+
   |1970-01-01T08:00:00.000+08:00|                                        15.0|                                        null|                                             10.0|
   |1970-01-01T08:00:00.100+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.200+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.300+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.400+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.500+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.600+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.700+08:00|                                         0.0|                                        null|                                              0.0|
   |1970-01-01T08:00:00.800+08:00|                                         0.0|                                        null|                                              0.0|
   +-----------------------------+--------------------------------------------+--------------------------------------------+-------------------------------------------------+
   Total line number = 9
   It costs 0.058s
   ```


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



[GitHub] [iotdb] wshao08 commented on pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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


   > By the way, the order of the results is not same with the sql statement. Is it normal?
   
   No, the printed column order is wrong. I will correct it soon.


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



[GitHub] [iotdb] wshao08 commented on a change in pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
##########
@@ -876,6 +877,16 @@ private void deduplicate(QueryPlan queryPlan, int fetchSize)
                   .getContext()
                   .getColumnName()
               : columnForReader;
+      if (queryPlan instanceof AggregationPlan && ((AggregationPlan) queryPlan).getLevel() >= 0) {
+        String aggregatePath =
+            originalPath.isMeasurementAliasExists()
+                ? FilePathUtils.generatePartialPathByLevel(
+                    originalPath.getFullPathWithAlias(), ((AggregationPlan) queryPlan).getLevel())
+                : FilePathUtils.generatePartialPathByLevel(
+                    originalPath.toString(), ((AggregationPlan) queryPlan).getLevel());

Review comment:
       The modified value of `columnForReader` is used between line 858-871. Considering the complexity of `deduplicate()`, I tried not to make much changes here to keep correctness. 
   Perhaps refactor `deduplicate()` and moving it into QueryPlan would be a choice. What do you think?




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

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



[GitHub] [iotdb] Alima777 commented on a change in pull request #2714: [IOTDB-1024] Support multiple aggregated measurements for group by level statement

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



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
##########
@@ -876,6 +877,16 @@ private void deduplicate(QueryPlan queryPlan, int fetchSize)
                   .getContext()
                   .getColumnName()
               : columnForReader;
+      if (queryPlan instanceof AggregationPlan && ((AggregationPlan) queryPlan).getLevel() >= 0) {
+        String aggregatePath =
+            originalPath.isMeasurementAliasExists()
+                ? FilePathUtils.generatePartialPathByLevel(
+                    originalPath.getFullPathWithAlias(), ((AggregationPlan) queryPlan).getLevel())
+                : FilePathUtils.generatePartialPathByLevel(
+                    originalPath.toString(), ((AggregationPlan) queryPlan).getLevel());

Review comment:
       Okay...but maybe we have to rewrite this method later...it's so long




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