You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2021/08/19 05:42:06 UTC

[iotdb] branch rel/0.12 updated: [IOTDB-1575] [To rel/0.12] Tsfile valueFilter cannot handle multiple pages (#3780)

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

haonan 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 d07b73c  [IOTDB-1575] [To rel/0.12] Tsfile valueFilter cannot handle multiple pages (#3780)
d07b73c is described below

commit d07b73c33ecee84ff8b971102699087670d86331
Author: Dawei Liu <li...@apache.org>
AuthorDate: Thu Aug 19 13:41:42 2021 +0800

    [IOTDB-1575] [To rel/0.12] Tsfile valueFilter cannot handle multiple pages (#3780)
---
 .../read/query/timegenerator/node/LeafNode.java    |  2 +-
 .../iotdb/tsfile/read/ReadOnlyTsFileTest.java      | 69 ++++++++++++++++++++++
 2 files changed, 70 insertions(+), 1 deletion(-)

diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java
index 4b68d65..a0b9d69 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/LeafNode.java
@@ -48,7 +48,7 @@ public class LeafNode implements Node {
       hasCached = true;
       return true;
     }
-    if (reader.hasNextBatch()) {
+    while (reader.hasNextBatch()) {
       cacheData = reader.nextBatch();
       if (cacheData.hasCurrent()) {
         cachedTime = cacheData.currentTime();
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/ReadOnlyTsFileTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/ReadOnlyTsFileTest.java
index 587fe18..b884fba 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/ReadOnlyTsFileTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/ReadOnlyTsFileTest.java
@@ -18,7 +18,12 @@
  */
 package org.apache.iotdb.tsfile.read;
 
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
@@ -33,12 +38,19 @@ import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.TsFileGeneratorForTest;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.TSRecord;
+import org.apache.iotdb.tsfile.write.record.datapoint.IntDataPoint;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.Schema;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 
 public class ReadOnlyTsFileTest {
 
@@ -47,6 +59,63 @@ public class ReadOnlyTsFileTest {
   private ReadOnlyTsFile tsFile;
 
   @Test
+  public void multiPagesTest() throws IOException, WriteProcessException {
+    final String filePath = "target/multiPages.tsfile";
+
+    TSFileConfig tsFileConfig = TSFileDescriptor.getInstance().getConfig();
+    // make multi pages in one group
+    tsFileConfig.setMaxNumberOfPointsInPage(100);
+    tsFileConfig.setGroupSizeInByte(100 * 1024 * 1024);
+    File file = new File(filePath);
+    TsFileWriter tsFileWriter = new TsFileWriter(file, new Schema(), tsFileConfig);
+
+    Path path = new Path("t", "id");
+    tsFileWriter.registerTimeseries(
+        path, new MeasurementSchema("id", TSDataType.INT32, TSEncoding.PLAIN, CompressionType.LZ4));
+
+    for (int i = 0; i < 11000000; i++) {
+      TSRecord t = new TSRecord(i, "t");
+      if (i % 100 == 0) {
+        // Add a large max_value to the page statistics,
+        // and get a very large number of invalid pages when the query is executed
+        t.addTuple(new IntDataPoint("id", 9000001));
+      } else {
+        t.addTuple(new IntDataPoint("id", i));
+      }
+      tsFileWriter.write(t);
+    }
+    // make same value to filter
+    TSRecord t = new TSRecord(101011000000L, "t");
+    t.addTuple(new IntDataPoint("id", 8000001));
+    tsFileWriter.write(t);
+    tsFileWriter.flushAllChunkGroups();
+    tsFileWriter.close();
+
+    ReadOnlyTsFile readOnlyTsFile = new ReadOnlyTsFile(new TsFileSequenceReader(filePath));
+
+    SingleSeriesExpression filter = new SingleSeriesExpression(path, ValueFilter.eq(8000001));
+    QueryExpression queryExpression = QueryExpression.create(Arrays.asList(path), filter);
+    QueryDataSet query = readOnlyTsFile.query(queryExpression);
+
+    int i = 0;
+    Assert.assertTrue(query.hasNext());
+    while (query.hasNext()) {
+      RowRecord next = query.next();
+      if (i == 0) {
+        Assert.assertEquals(next.getTimestamp(), 8000001);
+        Assert.assertEquals(next.getFields().get(0).getIntV(), 8000001);
+        i++;
+      } else {
+        Assert.assertEquals(next.getTimestamp(), 101011000000L);
+        Assert.assertEquals(next.getFields().get(0).getIntV(), 8000001);
+      }
+    }
+
+    readOnlyTsFile.close();
+    file.delete();
+  }
+
+  @Test
   public void test1() throws IOException {
     TSFileDescriptor.getInstance().getConfig().setTimeEncoder("TS_2DIFF");
     int rowCount = 1000;