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 2021/11/14 07:17:57 UTC

[iotdb] branch new_vector updated: fix bugs in value filter

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

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


The following commit(s) were added to refs/heads/new_vector by this push:
     new a27704a  fix bugs in value filter
     new 875816b  Merge branch 'new_vector' of https://github.com/apache/iotdb into new_vector
a27704a is described below

commit a27704aa6ce01d030b6c0a71295b64f08cc95631
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Sun Nov 14 15:17:09 2021 +0800

    fix bugs in value filter
---
 .../iotdb/AlignedTimeseriesSessionExample.java     | 24 +++++++++++++++++-----
 .../apache/iotdb/jdbc/IoTDBDatabaseMetadata.java   |  3 +--
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  6 +++---
 .../apache/iotdb/db/metadata/path/AlignedPath.java |  5 +++++
 .../apache/iotdb/db/utils/QueryDataSetUtils.java   |  3 +--
 .../org/apache/iotdb/tsfile/read/common/Path.java  |  5 +++++
 .../tsfile/read/query/dataset/QueryDataSet.java    | 10 +++++++++
 .../read/reader/chunk/AlignedChunkReader.java      |  9 +++++---
 .../tsfile/read/reader/page/AlignedPageReader.java | 12 ++++++++---
 9 files changed, 59 insertions(+), 18 deletions(-)

diff --git a/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java b/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
index 19e2706..38e0749 100644
--- a/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
+++ b/example/session/src/main/java/org/apache/iotdb/AlignedTimeseriesSessionExample.java
@@ -73,9 +73,22 @@ public class AlignedTimeseriesSessionExample {
     //    insertTabletWithAlignedTimeseriesMethod2();
     //    insertNullableTabletWithAlignedTimeseries();
     //    insertTabletsWithAlignedTimeseries();
-    selectTest();
     session.executeNonQueryStatement("flush");
     selectTest();
+    selectWithValueFilterTest();
+    session.executeNonQueryStatement("delete from root.sg_1.d1.s1 where time <= 5");
+    System.out.println("execute sql delete from root.sg_1.d1.s1 where time <= 5");
+    selectTest();
+    selectWithValueFilterTest();
+    session.executeNonQueryStatement("delete from root.sg_1.d1.s2 where time <= 3");
+    System.out.println("execute sql delete from root.sg_1.d1.s2 where time <= 3");
+
+    selectTest();
+    selectWithValueFilterTest();
+    session.executeNonQueryStatement("delete from root.sg_1.d1.s1 where time <= 10");
+    System.out.println("execute sql delete from root.sg_1.d1.s1 where time <= 10");
+    selectTest();
+    selectWithValueFilterTest();
 
     //    selectWithValueFilterTest();
     //    selectWithGroupByTest();
@@ -120,7 +133,8 @@ public class AlignedTimeseriesSessionExample {
   private static void selectWithValueFilterTest()
       throws StatementExecutionException, IoTDBConnectionException {
     SessionDataSet dataSet =
-        session.executeQueryStatement("select s1 from root.sg_1.d1.vector where s1 > 0");
+        session.executeQueryStatement("select s1 from root.sg_1.d1 where s1 > 3 and time < 9");
+    System.out.println(dataSet.getColumnNames());
     while (dataSet.hasNext()) {
       System.out.println(dataSet.next());
     }
@@ -128,7 +142,7 @@ public class AlignedTimeseriesSessionExample {
     dataSet.closeOperationHandle();
     dataSet =
         session.executeQueryStatement(
-            "select * from root.sg_1.d1.vector where time > 50 and s1 > 0 and s2 > 10000");
+            "select * from root.sg_1.d1 where time < 8 and s1 > 3 and s2 > 5");
     System.out.println(dataSet.getColumnNames());
     while (dataSet.hasNext()) {
       System.out.println(dataSet.next());
@@ -399,8 +413,8 @@ public class AlignedTimeseriesSessionExample {
 
     for (long time = 0; time < 10; time++) {
       List<Object> values = new ArrayList<>();
-      values.add(1L);
-      values.add(2);
+      values.add(time);
+      values.add((int) time);
       session.insertAlignedRecord(
           ROOT_SG1_D1_VECTOR1, time, multiMeasurementComponents, types, values);
     }
diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java
index 3c822ec..a96a807 100644
--- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java
+++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBDatabaseMetadata.java
@@ -512,8 +512,7 @@ public class IoTDBDatabaseMetadata implements DatabaseMetaData {
   public static TSQueryDataSet convertQueryDataSetByFetchSize(
       QueryDataSet queryDataSet, int fetchSize, WatermarkEncoder watermarkEncoder)
       throws IOException {
-    List<TSDataType> dataTypes = queryDataSet.getDataTypes();
-    int columnNum = dataTypes.size();
+    int columnNum = queryDataSet.getColumnNum();
     TSQueryDataSet tsQueryDataSet = new TSQueryDataSet();
     // one time column and each value column has a actual value buffer and a bitmap value to
     // indicate whether it is a null
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 84c9cfe..5c06283 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -344,13 +344,13 @@ public class IoTDBConfig {
   private int avgSeriesPointNumberThreshold = 10000;
 
   /** Only compact the sequence files */
-  private boolean enableSeqSpaceCompaction = true;
+  private boolean enableSeqSpaceCompaction = false;
 
   /** Only compact the unsequence files */
-  private boolean enableUnseqSpaceCompaction = true;
+  private boolean enableUnseqSpaceCompaction = false;
 
   /** Compact the unsequence files into the overlapped sequence files */
-  private boolean enableCrossSpaceCompaction = true;
+  private boolean enableCrossSpaceCompaction = false;
 
   /**
    * The strategy of inner space compaction task. There are just one inner space compaction strategy
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
index 997996b..c77697e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
@@ -401,4 +401,9 @@ public class AlignedPath extends PartialPath {
     chunkMetadataList.removeIf(context::chunkNotSatisfy);
     return new ArrayList<>(chunkMetadataList);
   }
+
+  @Override
+  public int getColumnNum() {
+    return measurementList.size();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
index f467a14..34c1483 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java
@@ -47,8 +47,7 @@ public class QueryDataSetUtils {
   public static TSQueryDataSet convertQueryDataSetByFetchSize(
       QueryDataSet queryDataSet, int fetchSize, WatermarkEncoder watermarkEncoder)
       throws IOException {
-    List<TSDataType> dataTypes = queryDataSet.getDataTypes();
-    int columnNum = dataTypes.size();
+    int columnNum = queryDataSet.getColumnNum();
     TSQueryDataSet tsQueryDataSet = new TSQueryDataSet();
     // one time column and each value column has a actual value buffer and a bitmap value to
     // indicate whether it is a null
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java
index 2ebae9d..3d7e7a9 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Path.java
@@ -165,4 +165,9 @@ public class Path implements Serializable, Comparable<Path> {
   public Path clone() {
     return new Path(fullPath);
   }
+
+  /** return the column contained by this path */
+  public int getColumnNum() {
+    return 1;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/QueryDataSet.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/QueryDataSet.java
index 0aaed56..10ccf06 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/QueryDataSet.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/QueryDataSet.java
@@ -48,6 +48,8 @@ public abstract class QueryDataSet {
   /** Only if all columns are null, we don't need that row */
   protected boolean withoutAllNull;
 
+  protected int columnNum;
+
   /** For redirect query. Need keep consistent with EndPoint in rpc.thrift. */
   public static class EndPoint {
     private String ip = null;
@@ -97,6 +99,10 @@ public abstract class QueryDataSet {
     this.paths = paths;
     this.dataTypes = dataTypes;
     this.ascending = ascending;
+    this.columnNum = 0;
+    for (Path p : paths) {
+      columnNum += p.getColumnNum();
+    }
   }
 
   public boolean hasNext() throws IOException {
@@ -198,4 +204,8 @@ public abstract class QueryDataSet {
   public void decreaseAlreadyReturnedRowNum() {
     alreadyReturnedRowNum--;
   }
+
+  public int getColumnNum() {
+    return columnNum;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/AlignedChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/AlignedChunkReader.java
index a9d3c67..46080ee 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/AlignedChunkReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/AlignedChunkReader.java
@@ -126,7 +126,7 @@ public class AlignedChunkReader implements IChunkReader {
         }
       }
       // if the current page satisfies
-      if (exits && pageSatisfied(timePageHeader)) {
+      if (exits && timePageSatisfied(timePageHeader)) {
         AlignedPageReader alignedPageReader =
             constructPageReaderForNextPage(timePageHeader, valuePageHeaderList);
         if (alignedPageReader != null) {
@@ -138,8 +138,11 @@ public class AlignedChunkReader implements IChunkReader {
     }
   }
 
-  private boolean pageSatisfied(PageHeader pageHeader) {
-    return filter == null || filter.satisfy(pageHeader.getStatistics());
+  /** used for time page filter */
+  private boolean timePageSatisfied(PageHeader timePageHeader) {
+    long startTime = timePageHeader.getStatistics().getStartTime();
+    long endTime = timePageHeader.getStatistics().getEndTime();
+    return filter == null || filter.satisfyStartEndTime(startTime, endTime);
   }
 
   /** used for value page filter */
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/AlignedPageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/AlignedPageReader.java
index 708e494..fd99337 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/AlignedPageReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/AlignedPageReader.java
@@ -84,6 +84,8 @@ public class AlignedPageReader implements IPageReader {
     }
     BatchData pageData = BatchDataFactory.createBatchData(TSDataType.VECTOR, ascending, false);
     boolean isNull;
+    // save the first not null value of each row
+    Object firstNotNullObject = null;
     for (int i = 0; i < timeBatch.length; i++) {
       // used to record whether the sub sensors are all null in current time
       isNull = true;
@@ -92,12 +94,14 @@ public class AlignedPageReader implements IPageReader {
         v[j] = valueBatchList.get(j) == null ? null : valueBatchList.get(j)[i];
         if (v[j] != null) {
           isNull = false;
+          firstNotNullObject = v[j].getValue();
         }
       }
       // if all the sub sensors' value are null in current time
       // or current row is not satisfied with the filter, just discard it
-      // TODO fix value filter v[0].getValue()
-      if (!isNull && (filter == null || filter.satisfy(timeBatch[i], v[0].getValue()))) {
+      // TODO fix value filter firstNotNullObject, currently, if it's a value filter, it will only
+      // accept AlignedPath with only one sub sensor
+      if (!isNull && (filter == null || filter.satisfy(timeBatch[i], firstNotNullObject))) {
         pageData.putVector(timeBatch[i], v);
       }
     }
@@ -106,7 +110,9 @@ public class AlignedPageReader implements IPageReader {
 
   public void setDeleteIntervalList(List<List<TimeRange>> list) {
     for (int i = 0; i < valueCount; i++) {
-      valuePageReaderList.get(i).setDeleteIntervalList(list.get(i));
+      if (valuePageReaderList.get(i) != null) {
+        valuePageReaderList.get(i).setDeleteIntervalList(list.get(i));
+      }
     }
   }