You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2022/08/24 01:35:06 UTC

[iotdb] branch rel/0.12 updated: [To rel/0.12] [IOTDB-4215] Fix incorrect aggregate query results due to wrong unseq file traversal order

This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch rel/0.12
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/0.12 by this push:
     new 05bc0168d6 [To rel/0.12] [IOTDB-4215] Fix incorrect aggregate query results due to wrong unseq file traversal order
05bc0168d6 is described below

commit 05bc0168d6765bd9884a02b13527be185d115646
Author: liuminghui233 <36...@users.noreply.github.com>
AuthorDate: Wed Aug 24 09:35:01 2022 +0800

    [To rel/0.12] [IOTDB-4215] Fix incorrect aggregate query results due to wrong unseq file traversal order
---
 .../db/query/executor/AggregationExecutor.java     |   3 +
 .../java/org/apache/iotdb/db/utils/QueryUtils.java |   2 +-
 .../aggregation/IoTDBAggregationScanOrderIT.java   | 132 +++++++++++++++++++++
 3 files changed, 136 insertions(+), 1 deletion(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
index fce0783071..c8977910a9 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
@@ -42,6 +42,7 @@ import org.apache.iotdb.db.query.reader.series.SeriesAggregateReader;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderByTimestamp;
 import org.apache.iotdb.db.query.timegenerator.ServerTimeGenerator;
 import org.apache.iotdb.db.utils.AggregateUtils;
+import org.apache.iotdb.db.utils.QueryUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -211,6 +212,7 @@ public class AggregationExecutor {
     timeFilter = queryDataSource.updateFilterUsingTTL(timeFilter);
 
     if (ascAggregateResultList != null && !ascAggregateResultList.isEmpty()) {
+      QueryUtils.fillOrderIndexes(queryDataSource, seriesPath.getDevice(), true);
       IAggregateReader seriesReader =
           new SeriesAggregateReader(
               seriesPath,
@@ -225,6 +227,7 @@ public class AggregationExecutor {
       aggregateFromReader(seriesReader, ascAggregateResultList);
     }
     if (descAggregateResultList != null && !descAggregateResultList.isEmpty()) {
+      QueryUtils.fillOrderIndexes(queryDataSource, seriesPath.getDevice(), false);
       IAggregateReader seriesReader =
           new SeriesAggregateReader(
               seriesPath,
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
index 29fab5acdc..dc02038344 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
@@ -99,7 +99,7 @@ public class QueryUtils {
   public static void fillOrderIndexes(
       QueryDataSource dataSource, String deviceId, boolean ascending) {
     List<TsFileResource> unseqResources = dataSource.getUnseqResources();
-    int[] orderIndex = new int[unseqResources.size() + 1];
+    int[] orderIndex = new int[unseqResources.size()];
     AtomicInteger index = new AtomicInteger();
     Map<Integer, Long> intToOrderTimeMap =
         unseqResources.stream()
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationScanOrderIT.java b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationScanOrderIT.java
new file mode 100644
index 0000000000..894ec9fc43
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/aggregation/IoTDBAggregationScanOrderIT.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.integration.aggregation;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.compaction.CompactionStrategy;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import static org.apache.iotdb.db.constant.TestConstant.first_value;
+import static org.apache.iotdb.db.constant.TestConstant.last_value;
+import static org.junit.Assert.fail;
+
+public class IoTDBAggregationScanOrderIT {
+
+  private static final String[] sqls =
+      new String[] {
+        "insert into root.sg1.d1(time, s1) values (12, 12);",
+        "flush;",
+        "insert into root.sg1.d1(time, s2) values (30, 30);",
+        "flush;",
+        "insert into root.sg1.d1(time, s1) values (0, 0);",
+        "insert into root.sg1.d1(time, s1) values (8, 8);",
+        "flush;",
+        "insert into root.sg1.d1(time, s1) values (0, 0);",
+        "insert into root.sg1.d1(time, s1) values (10, 10);",
+        "flush;",
+        "insert into root.sg1.d1(time, s1) values (17, 17);",
+        "insert into root.sg1.d1(time, s1) values (20, 20);",
+        "flush;"
+      };
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setCompactionStrategy(CompactionStrategy.NO_COMPACTION);
+    insertSQL();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setCompactionStrategy(CompactionStrategy.LEVEL_COMPACTION);
+  }
+
+  @Test
+  public void test() throws ClassNotFoundException {
+    String expectedRet = "0.0,20.0";
+    String d1s1 = "root.sg1.d1.s1";
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      boolean hasResultSet =
+          statement.execute("select first_value(s1), last_value(s1) from root.sg1.d1;");
+
+      Assert.assertTrue(hasResultSet);
+      try (ResultSet resultSet = statement.getResultSet()) {
+        Assert.assertTrue(resultSet.next());
+        String ans =
+            resultSet.getString(first_value(d1s1)) + "," + resultSet.getString(last_value(d1s1));
+        Assert.assertEquals(expectedRet, ans);
+        Assert.assertFalse(resultSet.next());
+      }
+
+      hasResultSet =
+          statement.execute(
+              "select first_value(s1), last_value(s1) from root.sg1.d1 order by time desc;");
+
+      Assert.assertTrue(hasResultSet);
+      try (ResultSet resultSet = statement.getResultSet()) {
+        Assert.assertTrue(resultSet.next());
+        String ans =
+            resultSet.getString(first_value(d1s1)) + "," + resultSet.getString(last_value(d1s1));
+        Assert.assertEquals(expectedRet, ans);
+        Assert.assertFalse(resultSet.next());
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  public static void insertSQL() throws ClassNotFoundException {
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      for (String sql : sqls) {
+        statement.execute(sql);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+}