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/03/15 01:19:17 UTC

[iotdb] branch TYQuery updated (ec70119 -> 2617771)

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

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


    from ec70119  format
     new 6cd892d  add VectorSeriesReader
     new b8b9477  format code
     new 9e466ca  need haonan
     new b7fe308  add todo
     add 3f6b915  [IOTDB-1225] Support CreateAlignedTimeseriesPlan in MManager
     add 96f29bb  [To Vector] add VectorPartialPath (#2823)
     add 6430a79  [To Vector] Vector TVList Step 1 (#2825)
     add b2ee6ca  [To Vector] flush vector series (#2826)
     add 567491f  Merge remote-tracking branch 'origin/Vector' into Vector
     add 14f8046  [IOTDB-1227] Support deleting aligned timeseries in MManager
     add 19e1aba  device template
     add 0198f5c  Add generated-sources as source folder
     add bfa529b  (Fix Github Action)Update apt source before installing libboost
     add 19ad435  [IOTDB-1204] set parameter in iotdb-cluster.properties (#2797)
     add 1f54e62  Update cpp client build doc (#2796)
     add 7d5329f  [IOTDB-1192] Fix sql end with semicolon  (#2798)
     add a857820  Update UDF User Defined Function.md
     add aad1bcd  [IOTDB-1193] Remove redundant sync meta leader in CManager for cluster module (#2787)
     add 02a4ecf  [IOTDB-1208] revise the cluster setup docs (#2807)
     add ff3a0f9  Fix compaction with cluster snapshot deletion (#2811)
     add 1f98174  [IOTDB-1221] Compaction module: chunk metadata lists returned by the getMeasurementChunkMetadataListMapIterator method are not lexicographically ordered by the measurement names (#2817)
     add bccb441  check and persist the system properties of the cluster version (#2816)
     add 251c979  Rename timeout parameter (#2824)
     add 653841b  download calcite-core's dependency penhato from public.nexus.pentaho.org rather than spring.io
     add 1b9d1f2  Merge branch 'master' into Vector
     add 6767feb  Merge branch 'Vector' of https://github.com/apache/iotdb into Vector
     add 6fed4ed  update template
     add fb51b68  [To Vector] [IOTDB-1225] Support CreateAlignedTimeseriesPlan in MManager (#2831)
     add a9d6687  [To Vector] change the way of flushing memtable from by column to by row (#2832)
     add 2e667d3  Merge remote-tracking branch 'origin/Vector' into Vector
     add b692da0  Skip code coverage and sonar reports in a forked pull request
     add 9ef4b81  Merge branch 'Vector' of https://github.com/apache/iotdb into Vector
     add ffeb7fe  Merge branch 'Vector' of https://github.com/apache/iotdb into Vector
     add 98fd47c  support insert vector with tablet plan
     add fe23fad  fix write vector bug
     add d6d821f  add serialize
     add 292bc65  update template
     add e149592  Merge branch 'Vector' of https://github.com/apache/iotdb into Vector
     add 0867605  Merge branch 'Vector' of https://github.com/apache/iotdb into Vector
     add bbcadd9  Merge remote-tracking branch 'origin/Vector' into Vector
     add f39bc42  [IOTDB-1228] Refactor PlanExecutor.insertTablet method to support aligned timeseries
     add 0aa64f5  Device template
     add 4d99c0b  vector mem control
     add 3601d1a  [To Vector][IOTDB-1228] Refactor PlanExecutor.insertTablet method to support aligned timeseries (#2833)
     add d9dd825  Merge branch 'Vector' of https://github.com/apache/iotdb into Vector
     add aab52e4  fix write vector bugssss
     add fe5a6a2  add .align
     add 62a5c31  spotless
     add e178a76  add header
     add cee50d4  Merge remote-tracking branch 'origin/Vector' into Vector
     add 932a408  Quick fix for experiment
     new 2617771  fix conflicts and format code

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .github/workflows/client.yml                       |  30 +-
 .github/workflows/sonar-coveralls.yml              |  14 +-
 antlr/pom.xml                                      |  19 +
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |   2 +-
 client-cpp/README.md                               |   6 +-
 .../resources/conf/iotdb-cluster.properties        |  10 +-
 .../java/org/apache/iotdb/cluster/ClusterMain.java |   7 +
 .../cluster/client/async/AsyncClientPool.java      |  10 +-
 .../iotdb/cluster/client/sync/SyncClientPool.java  |  10 +-
 .../apache/iotdb/cluster/config/ClusterConfig.java |  15 +
 .../iotdb/cluster/config/ClusterDescriptor.java    |  11 +
 .../iotdb/cluster/coordinator/Coordinator.java     |  27 +-
 .../apache/iotdb/cluster/metadata/CMManager.java   |  40 +-
 .../cluster/query/ClusterPhysicalGenerator.java    |  20 +-
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |   4 +
 .../cluster/query/reader/ClusterReaderFactory.java |   3 +-
 docs/UserGuide/Operation Manual/Kill Query.md      |   2 +-
 .../Operation Manual/UDF User Defined Function.md  |   2 +-
 docs/UserGuide/Server/Cluster Setup.md             |   4 +-
 docs/zh/UserGuide/Operation Manual/Kill Query.md   |   2 +-
 .../Operation Manual/UDF User Defined Function.md  |   2 +-
 docs/zh/UserGuide/Server/Cluster Setup.md          |   4 +-
 example/client-cpp-example/README.md               |   2 +-
 .../apache/iotdb/hadoop/tsfile/TsFileHelper.java   |   3 +-
 hive-connector/pom.xml                             |  12 +-
 .../resources/conf/iotdb-engine.properties         |   2 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  10 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   4 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   3 +-
 .../db/engine/compaction/TsFileManagement.java     |   8 +
 .../level/LevelCompactionTsFileManagement.java     |  31 ++
 .../engine/compaction/utils/CompactionUtils.java   |   3 +
 .../iotdb/db/engine/flush/MemTableFlushTask.java   |  59 ++-
 .../iotdb/db/engine/memtable/AbstractMemTable.java |  52 +-
 .../apache/iotdb/db/engine/memtable/IMemTable.java |  10 +
 .../db/engine/memtable/IWritableMemChunk.java      |   4 +
 .../db/engine/memtable/PrimitiveMemTable.java      |   5 +
 .../iotdb/db/engine/memtable/WritableMemChunk.java |  17 +
 .../db/engine/querycontext/QueryDataSource.java    |  12 +-
 .../db/engine/querycontext/ReadOnlyMemChunk.java   |   5 +
 .../engine/storagegroup/StorageGroupProcessor.java |  34 +-
 .../db/engine/storagegroup/TsFileProcessor.java    | 142 ++++--
 .../db/engine/storagegroup/TsFileResource.java     |  10 +-
 ...eption.java => AlignedTimeseriesException.java} |   9 +-
 .../org/apache/iotdb/db/metadata/MManager.java     | 308 ++++++++++--
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 301 ++++++++++--
 .../org/apache/iotdb/db/metadata/MetaUtils.java    |  16 +
 .../iotdb/db/metadata/logfile/MLogWriter.java      |  10 +
 .../org/apache/iotdb/db/metadata/mnode/MNode.java  |  12 +
 .../iotdb/db/metadata/template/Template.java       |  95 ++++
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  29 +-
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   4 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  14 +-
 .../db/qp/physical/crud/CreateTemplatePlan.java    | 239 +++++++++
 .../db/qp/physical/crud/InsertTabletPlan.java      |   6 +-
 .../SetDeviceTemplatePlan.java}                    |  83 ++--
 .../physical/sys/CreateAlignedTimeSeriesPlan.java  |   3 -
 .../iotdb/db/query/context/QueryContext.java       |   4 +-
 .../iotdb/db/query/control/QueryTimeManager.java   |   2 +-
 .../chunk/metadata/DiskChunkMetadataLoader.java    |   5 +
 .../chunk/metadata/MemChunkMetadataLoader.java     |   5 +
 .../query/reader/series/SeriesAggregateReader.java |   2 +-
 .../reader/series/SeriesRawDataBatchReader.java    |   4 +-
 .../iotdb/db/query/reader/series/SeriesReader.java | 115 ++---
 .../reader/series/SeriesReaderByTimestamp.java     |   4 +-
 .../query/reader/series/SeriesReaderFactory.java   |  91 ++++
 .../db/query/reader/series/VectorSeriesReader.java | 140 ++++++
 .../iotdb/db/rescon/PrimitiveArrayManager.java     |   8 +
 .../apache/iotdb/db/rescon/TVListAllocator.java    |  13 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  79 ++-
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |   1 -
 .../iotdb/db/utils/datastructure/TVList.java       |  27 ++
 .../iotdb/db/utils/datastructure/VectorTVList.java | 539 +++++++++++++++++++++
 .../db/engine/memtable/MemTableFlushTaskTest.java  |  32 +-
 .../db/engine/memtable/MemTableTestUtils.java      |  58 +++
 .../db/engine/memtable/PrimitiveMemTableTest.java  |  73 +++
 .../storagegroup/StorageGroupProcessorTest.java    |  28 +-
 .../iotdb/db/engine/storagegroup/TTLTest.java      |   6 +-
 .../engine/storagegroup/TsFileProcessorTest.java   |  72 ++-
 .../iotdb/db/integration/IoTDBQueryDemoIT.java     |   8 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       | 163 ++++++-
 .../iotdb/db/metadata/MManagerImproveTest.java     |   2 +-
 .../iotdb/db/qp/physical/InsertTabletPlanTest.java | 166 +++++++
 .../db/qp/physical/PhysicalPlanSerializeTest.java  |  20 +-
 .../reader/series/SeriesAggregateReaderTest.java   |   6 +-
 .../reader/series/SeriesReaderByTimestampTest.java |   6 +-
 .../db/utils/datastructure/VectorTVListTest.java   | 105 ++++
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   1 +
 .../java/org/apache/iotdb/session/Session.java     |  54 +++
 .../apache/iotdb/session/SessionConnection.java    |  40 ++
 .../java/org/apache/iotdb/session/SessionUT.java   |  47 ++
 thrift/pom.xml                                     |  19 +
 thrift/src/main/thrift/rpc.thrift                  |  19 +
 .../tsfile/file/metadata/TimeseriesMetadata.java   |   4 +
 .../file/metadata/VectorTimeSeriesMetadata.java    |  29 +-
 .../tsfile/file/metadata/enums/TSDataType.java     |   3 +-
 .../file/metadata/statistics/Statistics.java       |   2 +
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |  15 +-
 .../read/controller/IChunkMetadataLoader.java      |   2 +
 .../apache/iotdb/tsfile/utils/TsPrimitiveType.java |   4 +-
 .../tsfile/write/chunk/ChunkGroupWriterImpl.java   |   3 +-
 .../apache/iotdb/tsfile/write/record/Tablet.java   |  26 +-
 .../write/schema/VectorMeasurementSchema.java      |   9 +-
 ...easurementChunkMetadataListMapIteratorTest.java |  65 ++-
 .../zeppelin/iotdb/IoTDBInterpreterTest.java       |   2 +-
 105 files changed, 3427 insertions(+), 502 deletions(-)
 copy server/src/main/java/org/apache/iotdb/db/exception/metadata/{IllegalParameterOfPathException.java => AlignedTimeseriesException.java} (76%)
 create mode 100644 server/src/main/java/org/apache/iotdb/db/metadata/template/Template.java
 create mode 100644 server/src/main/java/org/apache/iotdb/db/qp/physical/crud/CreateTemplatePlan.java
 copy server/src/main/java/org/apache/iotdb/db/qp/physical/{sys/SetTTLPlan.java => crud/SetDeviceTemplatePlan.java} (50%)
 create mode 100644 server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
 create mode 100644 server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
 create mode 100644 server/src/main/java/org/apache/iotdb/db/utils/datastructure/VectorTVList.java
 create mode 100644 server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java


[iotdb] 03/05: need haonan

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9e466ca06a9f038ca57e16bc919791e393a15085
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Sun Mar 14 19:52:08 2021 +0800

    need haonan
---
 .../org/apache/iotdb/db/engine/StorageEngine.java  |  4 +-
 .../iotdb/db/engine/memtable/AbstractMemTable.java | 13 +++
 .../apache/iotdb/db/engine/memtable/IMemTable.java |  9 ++
 .../db/engine/querycontext/QueryDataSource.java    | 11 +--
 .../db/engine/querycontext/ReadOnlyMemChunk.java   |  5 ++
 .../engine/storagegroup/StorageGroupProcessor.java | 99 ++++++++++------------
 .../db/engine/storagegroup/TsFileProcessor.java    | 95 +++++++++++++--------
 .../db/engine/storagegroup/TsFileResource.java     | 40 ++++-----
 .../org/apache/iotdb/db/metadata/MManager.java     | 65 ++++++++------
 .../iotdb/db/query/context/QueryContext.java       | 11 ++-
 .../chunk/metadata/DiskChunkMetadataLoader.java    |  5 ++
 .../chunk/metadata/MemChunkMetadataLoader.java     |  5 ++
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |  1 -
 .../storagegroup/StorageGroupProcessorTest.java    | 23 +++--
 .../iotdb/db/engine/storagegroup/TTLTest.java      | 34 ++++----
 .../engine/storagegroup/TsFileProcessorTest.java   | 42 ++++-----
 .../reader/series/SeriesAggregateReaderTest.java   | 20 ++---
 .../reader/series/SeriesReaderByTimestampTest.java |  1 -
 .../tsfile/file/metadata/TimeseriesMetadata.java   |  4 +
 .../file/metadata/VectorTimeSeriesMetadata.java    | 29 ++++---
 .../read/controller/IChunkMetadataLoader.java      |  2 +
 21 files changed, 285 insertions(+), 233 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index b774e52..2b5cd02 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -665,10 +665,8 @@ public class StorageEngine implements IService {
       throws StorageEngineException, QueryProcessException {
     PartialPath fullPath = (PartialPath) seriesExpression.getSeriesPath();
     PartialPath deviceId = fullPath.getDevicePath();
-    String measurementId = seriesExpression.getSeriesPath().getMeasurement();
     StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
-    return storageGroupProcessor.query(
-        deviceId, measurementId, context, filePathsManager, seriesExpression.getFilter());
+    return storageGroupProcessor.query(fullPath, context, filePathsManager, seriesExpression.getFilter());
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index 3538aa1..4d68a78 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -274,7 +274,20 @@ public abstract class AbstractMemTable implements IMemTable {
         measurement, dataType, encoding, chunkCopy, props, curSize, deletionList);
   }
 
+
+  // TODO BY HAONAN HOU
   @Override
+  public ReadOnlyMemChunk query(
+      String deviceId,
+      String measurement,
+      IMeasurementSchema schema,
+      long timeLowerBound,
+      List<TimeRange> deletionList)
+      throws IOException, QueryProcessException, MetadataException {
+    return null;
+  }
+
+    @Override
   public void delete(
       PartialPath originalPath, PartialPath devicePath, long startTimestamp, long endTimestamp) {
     Map<String, IWritableMemChunk> deviceMap = memTableMap.get(devicePath.getFullPath());
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
index b3fb16a..e9e9c33 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
@@ -94,6 +94,15 @@ public interface IMemTable {
       List<TimeRange> deletionList)
       throws IOException, QueryProcessException, MetadataException;
 
+
+  ReadOnlyMemChunk query(
+      String deviceId,
+      String measurement,
+      IMeasurementSchema schema,
+      long timeLowerBound,
+      List<TimeRange> deletionList)
+      throws IOException, QueryProcessException, MetadataException;
+
   /** putBack all the memory resources. */
   void clear();
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
index b3dfb98..a3fc963 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
@@ -19,16 +19,13 @@
 
 package org.apache.iotdb.db.engine.querycontext;
 
+import java.util.List;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
 
-import java.util.List;
-
 public class QueryDataSource {
-  private PartialPath seriesPath;
   private List<TsFileResource> seqResources;
   private List<TsFileResource> unseqResources;
 
@@ -36,18 +33,12 @@ public class QueryDataSource {
   private long dataTTL = Long.MAX_VALUE;
 
   public QueryDataSource(
-      PartialPath seriesPath,
       List<TsFileResource> seqResources,
       List<TsFileResource> unseqResources) {
-    this.seriesPath = seriesPath;
     this.seqResources = seqResources;
     this.unseqResources = unseqResources;
   }
 
-  public PartialPath getSeriesPath() {
-    return seriesPath;
-  }
-
   public List<TsFileResource> getSeqResources() {
     return seqResources;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
index 8236e15..c635968 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
@@ -59,6 +59,7 @@ public class ReadOnlyMemChunk {
 
   private int chunkDataSize;
 
+  // TODO BY HAONAN HOU
   public ReadOnlyMemChunk(
       String measurementUid,
       TSDataType dataType,
@@ -124,6 +125,9 @@ public class ReadOnlyMemChunk {
           case DOUBLE:
             statsByType.update(timeValuePair.getTimestamp(), timeValuePair.getValue().getDouble());
             break;
+          case VECTOR:
+            statsByType.update(timeValuePair.getTimestamp());
+            break;
           default:
             throw new QueryProcessException("Unsupported data type:" + dataType);
         }
@@ -143,6 +147,7 @@ public class ReadOnlyMemChunk {
     return !chunkPointReader.hasNextTimeValuePair();
   }
 
+  // TODO BY HAONAN HOU
   public ChunkMetadata getChunkMetaData() {
     return cachedMetaData;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index 1f9d80e..e67b691 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -18,6 +18,38 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
+import static org.apache.iotdb.db.engine.merge.task.MergeTask.MERGE_SUFFIX;
+import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.TEMP_SUFFIX;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -77,43 +109,9 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-
-import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR;
-import static org.apache.iotdb.db.engine.merge.task.MergeTask.MERGE_SUFFIX;
-import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.TEMP_SUFFIX;
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
-
 /**
  * For sequence data, a StorageGroupProcessor has some TsFileProcessors, in which there is only one
  * TsFileProcessor in the working status. <br>
@@ -1535,8 +1533,7 @@ public class StorageGroupProcessor {
 
   // TODO need a read lock, please consider the concurrency with flush manager threads.
   public QueryDataSource query(
-      PartialPath deviceId,
-      String measurementId,
+      PartialPath fullPath,
       QueryContext context,
       QueryFileManager filePathsManager,
       Filter timeFilter)
@@ -1547,8 +1544,7 @@ public class StorageGroupProcessor {
           getFileResourceListForQuery(
               tsFileManagement.getTsFileList(true),
               upgradeSeqFileList,
-              deviceId,
-              measurementId,
+              fullPath,
               context,
               timeFilter,
               true);
@@ -1556,12 +1552,11 @@ public class StorageGroupProcessor {
           getFileResourceListForQuery(
               tsFileManagement.getTsFileList(false),
               upgradeUnseqFileList,
-              deviceId,
-              measurementId,
+              fullPath,
               context,
               timeFilter,
               false);
-      QueryDataSource dataSource = new QueryDataSource(deviceId, seqResources, unseqResources);
+      QueryDataSource dataSource = new QueryDataSource(seqResources, unseqResources);
       // used files should be added before mergeLock is unlocked, or they may be deleted by
       // running merge
       // is null only in tests
@@ -1592,24 +1587,24 @@ public class StorageGroupProcessor {
   private List<TsFileResource> getFileResourceListForQuery(
       Collection<TsFileResource> tsFileResources,
       List<TsFileResource> upgradeTsFileResources,
-      PartialPath deviceId,
-      String measurementId,
+      PartialPath fullPath,
       QueryContext context,
       Filter timeFilter,
       boolean isSeq)
       throws MetadataException {
+    String deviceId = fullPath.getDevice();
 
     if (config.isDebugOn()) {
       DEBUG_LOGGER.info(
           "Path: {}.{}, get tsfile list: {} isSeq: {} timefilter: {}",
-          deviceId.getFullPath(),
-          measurementId,
+          deviceId,
+          fullPath.getMeasurement(),
           tsFileResources,
           isSeq,
           (timeFilter == null ? "null" : timeFilter));
     }
 
-    IMeasurementSchema schema = IoTDB.metaManager.getSeriesSchema(deviceId, measurementId);
+    IMeasurementSchema schema = IoTDB.metaManager.getSeriesSchema(fullPath);
 
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     long timeLowerBound =
@@ -1618,7 +1613,7 @@ public class StorageGroupProcessor {
 
     // for upgrade files and old files must be closed
     for (TsFileResource tsFileResource : upgradeTsFileResources) {
-      if (!tsFileResource.isSatisfied(deviceId.getFullPath(), timeFilter, isSeq, dataTTL)) {
+      if (!tsFileResource.isSatisfied(deviceId, timeFilter, isSeq, dataTTL)) {
         continue;
       }
       closeQueryLock.readLock().lock();
@@ -1630,7 +1625,7 @@ public class StorageGroupProcessor {
     }
 
     for (TsFileResource tsFileResource : tsFileResources) {
-      if (!tsFileResource.isSatisfied(deviceId.getFullPath(), timeFilter, isSeq, dataTTL)) {
+      if (!tsFileResource.isSatisfied(fullPath.getDevice(), timeFilter, isSeq, dataTTL)) {
         continue;
       }
       closeQueryLock.readLock().lock();
@@ -1641,11 +1636,9 @@ public class StorageGroupProcessor {
           tsFileResource
               .getUnsealedFileProcessor()
               .query(
-                  deviceId.getFullPath(),
-                  measurementId,
-                  schema.getType(),
-                  schema.getEncodingType(),
-                  schema.getProps(),
+                  deviceId,
+                  fullPath.getMeasurement(),
+                  schema,
                   context,
                   tsfileResourcesForQuery);
         }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 1e705ab..773e5dd 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -18,6 +18,16 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -57,28 +67,18 @@ import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.TSStatus;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorChunkMetadata;
 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.TimeRange;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedDeque;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 @SuppressWarnings("java:S1135") // ignore todos
 public class TsFileProcessor {
 
@@ -91,7 +91,9 @@ public class TsFileProcessor {
   private StorageGroupInfo storageGroupInfo;
   private TsFileProcessorInfo tsFileProcessorInfo;
 
-  /** sync this object in query() and asyncTryToFlush() */
+  /**
+   * sync this object in query() and asyncTryToFlush()
+   */
   private final ConcurrentLinkedDeque<IMemTable> flushingMemTables = new ConcurrentLinkedDeque<>();
 
   private List<Pair<Modification, IMemTable>> modsToMemtable = new ArrayList<>();
@@ -113,7 +115,9 @@ public class TsFileProcessor {
 
   private IMemTable workMemTable;
 
-  /** this callback is called before the workMemtable is added into the flushingMemTables. */
+  /**
+   * this callback is called before the workMemtable is added into the flushingMemTables.
+   */
   private final UpdateEndTimeCallBack updateLatestFlushTimeCallback;
 
   private WriteLogNode logNode;
@@ -220,9 +224,9 @@ public class TsFileProcessor {
    * the range [start, end)
    *
    * @param insertTabletPlan insert a tablet of a device
-   * @param start start index of rows to be inserted in insertTabletPlan
-   * @param end end index of rows to be inserted in insertTabletPlan
-   * @param results result array
+   * @param start            start index of rows to be inserted in insertTabletPlan
+   * @param end              end index of rows to be inserted in insertTabletPlan
+   * @param results          result array
    */
   public void insertTablet(
       InsertTabletPlan insertTabletPlan, int start, int end, TSStatus[] results)
@@ -637,7 +641,9 @@ public class TsFileProcessor {
     }
   }
 
-  /** put the working memtable into flushing list and set the working memtable to null */
+  /**
+   * put the working memtable into flushing list and set the working memtable to null
+   */
   public void asyncFlush() {
     flushQueryLock.writeLock().lock();
     if (logger.isDebugEnabled()) {
@@ -704,7 +710,9 @@ public class TsFileProcessor {
     FlushManager.getInstance().registerTsFileProcessor(this);
   }
 
-  /** put back the memtable to MemTablePool and make metadata in writer visible */
+  /**
+   * put back the memtable to MemTablePool and make metadata in writer visible
+   */
   private void releaseFlushedMemTable(IMemTable memTable) {
     flushQueryLock.writeLock().lock();
     if (logger.isDebugEnabled()) {
@@ -1043,18 +1051,14 @@ public class TsFileProcessor {
    * memtables and then compact them into one TimeValuePairSorter). Then get the related
    * ChunkMetadata of data on disk.
    *
-   * @param deviceId device id
+   * @param deviceId      device id
    * @param measurementId measurements id
-   * @param dataType data type
-   * @param encoding encoding
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public void query(
       String deviceId,
       String measurementId,
-      TSDataType dataType,
-      TSEncoding encoding,
-      Map<String, String> props,
+      IMeasurementSchema schema,
       QueryContext context,
       List<TsFileResource> tsfileResourcesForQuery)
       throws IOException, MetadataException {
@@ -1078,9 +1082,7 @@ public class TsFileProcessor {
             flushingMemTable.query(
                 deviceId,
                 measurementId,
-                dataType,
-                encoding,
-                props,
+                schema,
                 context.getQueryTimeLowerBound(),
                 deletionList);
         if (memChunk != null) {
@@ -1092,9 +1094,7 @@ public class TsFileProcessor {
             workMemTable.query(
                 deviceId,
                 measurementId,
-                dataType,
-                encoding,
-                props,
+                schema,
                 context.getQueryTimeLowerBound(),
                 null);
         if (memChunk != null) {
@@ -1108,8 +1108,33 @@ public class TsFileProcessor {
               modificationFile,
               new PartialPath(deviceId + IoTDBConstant.PATH_SEPARATOR + measurementId));
 
-      List<ChunkMetadata> chunkMetadataList =
-          writer.getVisibleMetadataList(deviceId, measurementId, dataType);
+      List<IChunkMetadata> chunkMetadataList = new ArrayList<>();
+      if (schema instanceof VectorMeasurementSchema) {
+        List<ChunkMetadata> timeChunkMetadataList =
+            writer.getVisibleMetadataList(deviceId, measurementId, schema.getType());
+        List<List<ChunkMetadata>> valueChunkMetadataList = new ArrayList<>();
+        List<String> valueMeasurementIdList = schema.getValueMeasurementIdList();
+        List<TSDataType> valueDataTypeList = schema.getValueTSDataTypeList();
+        for (int i = 0; i < valueMeasurementIdList.size(); i++) {
+          valueChunkMetadataList.add(writer
+              .getVisibleMetadataList(deviceId, valueMeasurementIdList.get(i),
+                  valueDataTypeList.get(i)));
+        }
+
+
+        for (int i = 0; i < timeChunkMetadataList.size(); i++) {
+          List<IChunkMetadata> valueChunkMetadata = new ArrayList<>();
+          for (List<ChunkMetadata> chunkMetadata : valueChunkMetadataList) {
+            valueChunkMetadata.add(chunkMetadata.get(i));
+          }
+          chunkMetadataList
+              .add(new VectorChunkMetadata(timeChunkMetadataList.get(i), valueChunkMetadata));
+        }
+      } else {
+        chunkMetadataList =
+            new ArrayList<>(writer.getVisibleMetadataList(deviceId, measurementId, schema.getType()));
+      }
+
       QueryUtils.modifyChunkMetaData(chunkMetadataList, modifications);
       chunkMetadataList.removeIf(context::chunkNotSatisfy);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
index 6d4f58b..57472f0 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
@@ -18,6 +18,20 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Random;
+import java.util.Set;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
@@ -32,7 +46,6 @@ import org.apache.iotdb.db.service.UpgradeSevice;
 import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -41,25 +54,9 @@ import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.FileAlreadyExistsException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Random;
-import java.util.Set;
-
 @SuppressWarnings("java:S1135") // ignore todos
 public class TsFileResource {
 
@@ -103,7 +100,7 @@ public class TsFileResource {
    * Chunk metadata list of unsealed tsfile. Only be set in a temporal TsFileResource in a query
    * process.
    */
-  private List<ChunkMetadata> chunkMetadataList;
+  private List<IChunkMetadata> chunkMetadataList;
 
   /** Mem chunk data. Only be set in a temporal TsFileResource in a query process. */
   private List<ReadOnlyMemChunk> readOnlyMemChunk;
@@ -184,7 +181,7 @@ public class TsFileResource {
   /** unsealed TsFile */
   public TsFileResource(
       List<ReadOnlyMemChunk> readOnlyMemChunk,
-      List<ChunkMetadata> chunkMetadataList,
+      List<IChunkMetadata> chunkMetadataList,
       TsFileResource originTsFileResource)
       throws IOException {
     this.file = originTsFileResource.file;
@@ -206,7 +203,6 @@ public class TsFileResource {
   }
 
   private void generateTimeSeriesMetadata() throws IOException {
-    timeSeriesMetadata = new TimeseriesMetadata();
     timeSeriesMetadata.setOffsetOfChunkMetaDataList(-1);
     timeSeriesMetadata.setDataSizeOfChunkMetaDataList(-1);
 
@@ -223,7 +219,7 @@ public class TsFileResource {
       Statistics<?> seriesStatistics =
           Statistics.getStatsByType(timeSeriesMetadata.getTSDataType());
       // flush chunkMetadataList one by one
-      for (ChunkMetadata chunkMetadata : chunkMetadataList) {
+      for (IChunkMetadata chunkMetadata : chunkMetadataList) {
         seriesStatistics.mergeStatistics(chunkMetadata.getStatistics());
       }
 
@@ -234,7 +230,7 @@ public class TsFileResource {
       }
       timeSeriesMetadata.setStatistics(seriesStatistics);
     } else {
-      timeSeriesMetadata = null;
+      this.timeSeriesMetadata = null;
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 76703a8..f6c6974 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -18,6 +18,32 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import static java.util.stream.Collectors.toList;
+import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
@@ -73,37 +99,9 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
-
 /**
  * This class takes the responsibility of serialization of all the metadata info and persistent it
  * into files. This class contains all the interfaces to modify the metadata for delta system. All
@@ -1015,6 +1013,17 @@ public class MManager {
     return null;
   }
 
+  // TODO BY ZESONG SUN
+  public IMeasurementSchema getSeriesSchema(PartialPath fullPath)
+      throws MetadataException {
+    MNode node = mtree.getNodeByPath(fullPath.getDevicePath());
+    MNode leaf = node.getChild(fullPath.getMeasurement());
+    if (leaf != null) {
+      return ((MeasurementMNode) leaf).getSchema();
+    }
+    return null;
+  }
+
   /**
    * Get child node path in the next level of the given path.
    *
diff --git a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index 3febfa9..0c53d08 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@ -19,16 +19,15 @@
 
 package org.apache.iotdb.db.query.context;
 
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.modification.ModificationFile;
-import org.apache.iotdb.db.metadata.PartialPath;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
 
 /** QueryContext contains the shared information with in a query. */
 public class QueryContext {
@@ -95,7 +94,7 @@ public class QueryContext {
     this.queryTimeLowerBound = queryTimeLowerBound;
   }
 
-  public boolean chunkNotSatisfy(ChunkMetadata chunkMetaData) {
+  public boolean chunkNotSatisfy(IChunkMetadata chunkMetaData) {
     return chunkMetaData.getEndTime() < queryTimeLowerBound;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
index 3a6fc2a..4ac0080 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/DiskChunkMetadataLoader.java
@@ -72,6 +72,11 @@ public class DiskChunkMetadataLoader implements IChunkMetadataLoader {
     return chunkMetadataList;
   }
 
+  @Override
+  public boolean isMemChunkMetadataLoader() {
+    return false;
+  }
+
   public static void setDiskChunkLoader(
       List<IChunkMetadata> chunkMetadataList,
       TsFileResource resource,
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java
index 8a22d0b..cf022e3 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/chunk/metadata/MemChunkMetadataLoader.java
@@ -73,4 +73,9 @@ public class MemChunkMetadataLoader implements IChunkMetadataLoader {
     }
     return chunkMetadataList;
   }
+
+  @Override
+  public boolean isMemChunkMetadataLoader() {
+    return true;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
index 050de31..0f9297a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
@@ -157,7 +157,6 @@ public class FileLoaderUtils {
    */
   public static List<IPageReader> loadPageReaderList(
       IChunkMetadata chunkMetaData, Filter timeFilter) throws IOException {
-    // TODO memory Vector chunk metadata
     if (chunkMetaData == null) {
       throw new IOException("Can't init null chunkMeta");
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
index e68c4af..6b64b13 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
@@ -38,6 +38,7 @@ import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
+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.TimeValuePair;
@@ -125,9 +126,7 @@ public class StorageGroupProcessorTest {
       tsfileProcessor.query(
           deviceId,
           measurementId,
-          TSDataType.INT32,
-          TSEncoding.RLE,
-          Collections.emptyMap(),
+          new MeasurementSchema(measurementId, TSDataType.INT32, TSEncoding.RLE, CompressionType.UNCOMPRESSED, Collections.emptyMap()),
           new QueryContext(),
           tsfileResourcesForQuery);
     }
@@ -162,7 +161,7 @@ public class StorageGroupProcessorTest {
     }
     processor.syncCloseAllWorkingTsFileProcessors();
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
     Assert.assertEquals(10, queryDataSource.getSeqResources().size());
     for (TsFileResource resource : queryDataSource.getSeqResources()) {
       Assert.assertTrue(resource.isClosed());
@@ -191,7 +190,7 @@ public class StorageGroupProcessorTest {
     processor.syncCloseAllWorkingTsFileProcessors();
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
     Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
   }
 
@@ -252,7 +251,7 @@ public class StorageGroupProcessorTest {
     processor.syncCloseAllWorkingTsFileProcessors();
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
 
     Assert.assertEquals(2, queryDataSource.getSeqResources().size());
     Assert.assertEquals(1, queryDataSource.getUnseqResources().size());
@@ -282,7 +281,7 @@ public class StorageGroupProcessorTest {
     processor.syncCloseAllWorkingTsFileProcessors();
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
     Assert.assertEquals(10, queryDataSource.getSeqResources().size());
     Assert.assertEquals(10, queryDataSource.getUnseqResources().size());
     for (TsFileResource resource : queryDataSource.getSeqResources()) {
@@ -322,7 +321,7 @@ public class StorageGroupProcessorTest {
     }
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
     Assert.assertEquals(10, queryDataSource.getSeqResources().size());
     Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
     for (TsFileResource resource : queryDataSource.getSeqResources()) {
@@ -404,7 +403,7 @@ public class StorageGroupProcessorTest {
     }
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
 
     Assert.assertEquals(2, queryDataSource.getSeqResources().size());
     Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
@@ -486,7 +485,7 @@ public class StorageGroupProcessorTest {
     }
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
 
     Assert.assertEquals(2, queryDataSource.getSeqResources().size());
     Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
@@ -568,7 +567,7 @@ public class StorageGroupProcessorTest {
     }
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
 
     Assert.assertEquals(2, queryDataSource.getSeqResources().size());
     Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
@@ -606,7 +605,7 @@ public class StorageGroupProcessorTest {
     }
 
     QueryDataSource queryDataSource =
-        processor.query(new PartialPath(deviceId), measurementId, context, null, null);
+        processor.query(new PartialPath(deviceId, measurementId), context, null, null);
     Assert.assertEquals(10, queryDataSource.getSeqResources().size());
     for (TsFileResource resource : queryDataSource.getSeqResources()) {
       Assert.assertTrue(resource.isClosed());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
index 1943ef3..a76f09c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
@@ -20,6 +20,19 @@
 
 package org.apache.iotdb.db.engine.storagegroup;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
@@ -54,25 +67,10 @@ import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.iotdb.tsfile.read.reader.IBatchReader;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TTLTest {
 
   private String sg1 = "root.TTL_SG1";
@@ -223,7 +221,7 @@ public class TTLTest {
     // files before ttl
     QueryDataSource dataSource =
         storageGroupProcessor.query(
-            new PartialPath(sg1), s1, EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
+            new PartialPath(sg1, s1), EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
     List<TsFileResource> seqResource = dataSource.getSeqResources();
     List<TsFileResource> unseqResource = dataSource.getUnseqResources();
     assertEquals(4, seqResource.size());
@@ -234,7 +232,7 @@ public class TTLTest {
     // files after ttl
     dataSource =
         storageGroupProcessor.query(
-            new PartialPath(sg1), s1, EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
+            new PartialPath(sg1, s1), EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
     seqResource = dataSource.getSeqResources();
     unseqResource = dataSource.getUnseqResources();
     assertTrue(seqResource.size() < 4);
@@ -269,7 +267,7 @@ public class TTLTest {
     storageGroupProcessor.setDataTTL(0);
     dataSource =
         storageGroupProcessor.query(
-            new PartialPath(sg1), s1, EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
+            new PartialPath(sg1, s1), EnvironmentUtils.TEST_QUERY_CONTEXT, null, null);
     seqResource = dataSource.getSeqResources();
     unseqResource = dataSource.getUnseqResources();
     assertEquals(0, seqResource.size());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index f972a83..276cacd 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@ -18,6 +18,15 @@
  */
 package org.apache.iotdb.db.engine.storagegroup;
 
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
@@ -30,30 +39,21 @@ import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+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.TimeValuePair;
 import org.apache.iotdb.tsfile.read.reader.IPointReader;
 import org.apache.iotdb.tsfile.write.record.TSRecord;
 import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import static junit.framework.TestCase.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
 public class TsFileProcessorTest {
 
   private TsFileProcessor processor;
@@ -104,7 +104,7 @@ public class TsFileProcessorTest {
     SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertTrue(tsfileResourcesForQuery.isEmpty());
 
     for (int i = 1; i <= 100; i++) {
@@ -116,7 +116,7 @@ public class TsFileProcessorTest {
     // query data in memory
     tsfileResourcesForQuery.clear();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
     List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
     for (ReadOnlyMemChunk chunk : memChunks) {
@@ -134,7 +134,7 @@ public class TsFileProcessorTest {
 
     tsfileResourcesForQuery.clear();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
     assertEquals(1, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
     assertEquals(
@@ -166,7 +166,7 @@ public class TsFileProcessorTest {
     SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertTrue(tsfileResourcesForQuery.isEmpty());
 
     for (int i = 1; i <= 100; i++) {
@@ -178,7 +178,7 @@ public class TsFileProcessorTest {
     // query data in memory
     tsfileResourcesForQuery.clear();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
     int num = 1;
     List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
@@ -197,7 +197,7 @@ public class TsFileProcessorTest {
 
     tsfileResourcesForQuery.clear();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
     assertEquals(1, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
     assertEquals(
@@ -254,7 +254,7 @@ public class TsFileProcessorTest {
     SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertTrue(tsfileResourcesForQuery.isEmpty());
 
     for (int flushId = 0; flushId < 10; flushId++) {
@@ -269,7 +269,7 @@ public class TsFileProcessorTest {
 
     tsfileResourcesForQuery.clear();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertFalse(tsfileResourcesForQuery.isEmpty());
     assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
     assertEquals(10, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
@@ -302,7 +302,7 @@ public class TsFileProcessorTest {
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
 
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertTrue(tsfileResourcesForQuery.isEmpty());
 
     for (int i = 1; i <= 100; i++) {
@@ -314,7 +314,7 @@ public class TsFileProcessorTest {
     // query data in memory
     tsfileResourcesForQuery.clear();
     processor.query(
-        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
+        deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
     assertFalse(tsfileResourcesForQuery.isEmpty());
     assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
     List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
index 55eea5c..a713157 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
@@ -19,6 +19,15 @@
 
 package org.apache.iotdb.db.query.reader.series;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -33,19 +42,10 @@ 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;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-
 public class SeriesAggregateReaderTest {
 
   private static final String SERIES_READER_TEST_SG = "root.seriesReaderTest";
@@ -71,7 +71,7 @@ public class SeriesAggregateReaderTest {
       PartialPath path = new PartialPath(SERIES_READER_TEST_SG + ".device0.sensor0");
       Set<String> allSensors = new HashSet<>();
       allSensors.add("sensor0");
-      QueryDataSource queryDataSource = new QueryDataSource(path, seqResources, unseqResources);
+      QueryDataSource queryDataSource = new QueryDataSource(seqResources, unseqResources);
       SeriesAggregateReader seriesReader =
           new SeriesAggregateReader(
               path,
diff --git a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
index 4e71afa..129f18d 100644
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
@@ -63,7 +63,6 @@ public class SeriesReaderByTimestampTest {
   public void test() throws IOException, IllegalPathException {
     QueryDataSource dataSource =
         new QueryDataSource(
-            new PartialPath(SERIES_READER_TEST_SG + ".device0.sensor0"),
             seqResources,
             unseqResources);
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
index 10efcb7..e86fcb2 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
@@ -189,6 +189,10 @@ public class TimeseriesMetadata implements Accountable, ITimeSeriesMetadata {
     this.chunkMetadataLoader = chunkMetadataLoader;
   }
 
+  public IChunkMetadataLoader getChunkMetadataLoader() {
+    return chunkMetadataLoader;
+  }
+
   @Override
   public List<IChunkMetadata> loadChunkMetadataList() throws IOException {
     return chunkMetadataLoader.loadChunkMetadataList(this);
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java
index 20a25df..bfa8e74 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/VectorTimeSeriesMetadata.java
@@ -62,23 +62,26 @@ public class VectorTimeSeriesMetadata implements ITimeSeriesMetadata {
 
   @Override
   public List<IChunkMetadata> loadChunkMetadataList() throws IOException {
-    List<IChunkMetadata> timeChunkMetadata = timeseriesMetadata.loadChunkMetadataList();
-    List<List<IChunkMetadata>> valueChunkMetadataList = new ArrayList<>();
-    for (TimeseriesMetadata metadata : valueTimeseriesMetadataList) {
-      valueChunkMetadataList.add(metadata.loadChunkMetadataList());
-    }
+    if (timeseriesMetadata.getChunkMetadataLoader().isMemChunkMetadataLoader()) {
+      return timeseriesMetadata.loadChunkMetadataList();
+    } else {
+      List<IChunkMetadata> timeChunkMetadata = timeseriesMetadata.loadChunkMetadataList();
+      List<List<IChunkMetadata>> valueChunkMetadataList = new ArrayList<>();
+      for (TimeseriesMetadata metadata : valueTimeseriesMetadataList) {
+        valueChunkMetadataList.add(metadata.loadChunkMetadataList());
+      }
 
-    List<IChunkMetadata> res = new ArrayList<>();
+      List<IChunkMetadata> res = new ArrayList<>();
 
-    for (int i = 0; i < timeChunkMetadata.size(); i++) {
-      List<IChunkMetadata> chunkMetadataList = new ArrayList<>();
-      for (List<IChunkMetadata> chunkMetadata : valueChunkMetadataList) {
-        chunkMetadataList.add(chunkMetadata.get(i));
+      for (int i = 0; i < timeChunkMetadata.size(); i++) {
+        List<IChunkMetadata> chunkMetadataList = new ArrayList<>();
+        for (List<IChunkMetadata> chunkMetadata : valueChunkMetadataList) {
+          chunkMetadataList.add(chunkMetadata.get(i));
+        }
+        res.add(new VectorChunkMetadata(timeChunkMetadata.get(i), chunkMetadataList));
       }
-      res.add(new VectorChunkMetadata(timeChunkMetadata.get(i), chunkMetadataList));
+      return res;
     }
-
-    return res;
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java
index 093219d..71371a4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IChunkMetadataLoader.java
@@ -29,4 +29,6 @@ public interface IChunkMetadataLoader {
   /** read all chunk metadata of one time series in one file. */
   List<IChunkMetadata> loadChunkMetadataList(ITimeSeriesMetadata timeseriesMetadata)
       throws IOException;
+
+  boolean isMemChunkMetadataLoader();
 }


[iotdb] 05/05: fix conflicts and format code

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2617771109efa81ec48b1be4217fb5d9a3552402
Merge: b7fe308 932a408
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Mon Mar 15 09:18:50 2021 +0800

    fix conflicts and format code

 .github/workflows/client.yml                       |  30 +-
 .github/workflows/sonar-coveralls.yml              |  14 +-
 antlr/pom.xml                                      |  19 +
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |   2 +-
 client-cpp/README.md                               |   6 +-
 .../resources/conf/iotdb-cluster.properties        |  10 +-
 .../java/org/apache/iotdb/cluster/ClusterMain.java |   7 +
 .../cluster/client/async/AsyncClientPool.java      |  10 +-
 .../iotdb/cluster/client/sync/SyncClientPool.java  |  10 +-
 .../apache/iotdb/cluster/config/ClusterConfig.java |  15 +
 .../iotdb/cluster/config/ClusterDescriptor.java    |  11 +
 .../iotdb/cluster/coordinator/Coordinator.java     |  27 +-
 .../apache/iotdb/cluster/metadata/CMManager.java   |  40 +-
 .../cluster/query/ClusterPhysicalGenerator.java    |  20 +-
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |   4 +
 docs/UserGuide/Operation Manual/Kill Query.md      |   2 +-
 .../Operation Manual/UDF User Defined Function.md  |   2 +-
 docs/UserGuide/Server/Cluster Setup.md             |   4 +-
 docs/zh/UserGuide/Operation Manual/Kill Query.md   |   2 +-
 .../Operation Manual/UDF User Defined Function.md  |   2 +-
 docs/zh/UserGuide/Server/Cluster Setup.md          |   4 +-
 example/client-cpp-example/README.md               |   2 +-
 .../apache/iotdb/hadoop/tsfile/TsFileHelper.java   |   3 +-
 hive-connector/pom.xml                             |  12 +-
 .../resources/conf/iotdb-engine.properties         |   2 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  10 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   4 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   3 +-
 .../db/engine/compaction/TsFileManagement.java     |   8 +
 .../level/LevelCompactionTsFileManagement.java     |  31 ++
 .../engine/compaction/utils/CompactionUtils.java   |   3 +
 .../iotdb/db/engine/flush/MemTableFlushTask.java   |  59 ++-
 .../iotdb/db/engine/memtable/AbstractMemTable.java |  41 +-
 .../apache/iotdb/db/engine/memtable/IMemTable.java |   3 +-
 .../db/engine/memtable/IWritableMemChunk.java      |   4 +
 .../db/engine/memtable/PrimitiveMemTable.java      |   5 +
 .../iotdb/db/engine/memtable/WritableMemChunk.java |  17 +
 .../db/engine/querycontext/QueryDataSource.java    |   7 +-
 .../engine/storagegroup/StorageGroupProcessor.java |  73 ++-
 .../db/engine/storagegroup/TsFileProcessor.java    | 157 +++---
 .../db/engine/storagegroup/TsFileResource.java     |  30 +-
 .../metadata/AlignedTimeseriesException.java       |  31 ++
 .../org/apache/iotdb/db/metadata/MManager.java     | 355 +++++++++++---
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 301 ++++++++++--
 .../org/apache/iotdb/db/metadata/MetaUtils.java    |  16 +
 .../iotdb/db/metadata/logfile/MLogWriter.java      |  10 +
 .../org/apache/iotdb/db/metadata/mnode/MNode.java  |  12 +
 .../iotdb/db/metadata/template/Template.java       |  95 ++++
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  29 +-
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   4 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  14 +-
 .../db/qp/physical/crud/CreateTemplatePlan.java    | 239 +++++++++
 .../db/qp/physical/crud/InsertTabletPlan.java      |   6 +-
 .../db/qp/physical/crud/SetDeviceTemplatePlan.java |  93 ++++
 .../physical/sys/CreateAlignedTimeSeriesPlan.java  |   3 -
 .../iotdb/db/query/context/QueryContext.java       |   9 +-
 .../iotdb/db/query/control/QueryTimeManager.java   |   2 +-
 .../iotdb/db/rescon/PrimitiveArrayManager.java     |   8 +
 .../apache/iotdb/db/rescon/TVListAllocator.java    |  13 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  79 ++-
 .../iotdb/db/utils/datastructure/TVList.java       |  27 ++
 .../iotdb/db/utils/datastructure/VectorTVList.java | 539 +++++++++++++++++++++
 .../db/engine/memtable/MemTableFlushTaskTest.java  |  32 +-
 .../db/engine/memtable/MemTableTestUtils.java      |  58 +++
 .../db/engine/memtable/PrimitiveMemTableTest.java  |  73 +++
 .../storagegroup/StorageGroupProcessorTest.java    |   7 +-
 .../iotdb/db/engine/storagegroup/TTLTest.java      |  28 +-
 .../engine/storagegroup/TsFileProcessorTest.java   |  90 +++-
 .../iotdb/db/integration/IoTDBQueryDemoIT.java     |   8 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       | 163 ++++++-
 .../iotdb/db/metadata/MManagerImproveTest.java     |   2 +-
 .../iotdb/db/qp/physical/InsertTabletPlanTest.java | 166 +++++++
 .../db/qp/physical/PhysicalPlanSerializeTest.java  |  20 +-
 .../reader/series/SeriesAggregateReaderTest.java   |  20 +-
 .../reader/series/SeriesReaderByTimestampTest.java |   5 +-
 .../db/utils/datastructure/VectorTVListTest.java   | 105 ++++
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   1 +
 .../java/org/apache/iotdb/session/Session.java     |  54 +++
 .../apache/iotdb/session/SessionConnection.java    |  40 ++
 .../java/org/apache/iotdb/session/SessionUT.java   |  47 ++
 thrift/pom.xml                                     |  19 +
 thrift/src/main/thrift/rpc.thrift                  |  19 +
 .../tsfile/file/metadata/enums/TSDataType.java     |   3 +-
 .../file/metadata/statistics/Statistics.java       |   2 +
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |  15 +-
 .../apache/iotdb/tsfile/utils/TsPrimitiveType.java |   4 +-
 .../tsfile/write/chunk/ChunkGroupWriterImpl.java   |   3 +-
 .../apache/iotdb/tsfile/write/record/Tablet.java   |  26 +-
 .../write/schema/VectorMeasurementSchema.java      |   9 +-
 ...easurementChunkMetadataListMapIteratorTest.java |  65 ++-
 .../zeppelin/iotdb/IoTDBInterpreterTest.java       |   2 +-
 91 files changed, 3234 insertions(+), 452 deletions(-)

diff --cc server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index 2b5cd02,b774e52..d2568db
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@@ -665,8 -665,10 +665,9 @@@ public class StorageEngine implements I
        throws StorageEngineException, QueryProcessException {
      PartialPath fullPath = (PartialPath) seriesExpression.getSeriesPath();
      PartialPath deviceId = fullPath.getDevicePath();
 -    String measurementId = seriesExpression.getSeriesPath().getMeasurement();
      StorageGroupProcessor storageGroupProcessor = getProcessor(deviceId);
-     return storageGroupProcessor.query(fullPath, context, filePathsManager, seriesExpression.getFilter());
+     return storageGroupProcessor.query(
 -        deviceId, measurementId, context, filePathsManager, seriesExpression.getFilter());
++        fullPath, context, filePathsManager, seriesExpression.getFilter());
    }
  
    /**
diff --cc server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index ff9659b,cb76ce6..ef7f545
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@@ -274,19 -292,6 +292,18 @@@ public abstract class AbstractMemTable 
          measurement, dataType, encoding, chunkCopy, props, curSize, deletionList);
    }
  
- 
 +  // TODO BY HAONAN HOU
 +  @Override
 +  public ReadOnlyMemChunk query(
 +      String deviceId,
 +      String measurement,
 +      IMeasurementSchema schema,
 +      long timeLowerBound,
 +      List<TimeRange> deletionList)
 +      throws IOException, QueryProcessException, MetadataException {
 +    return null;
 +  }
 +
    @Override
    public void delete(
        PartialPath originalPath, PartialPath devicePath, long startTimestamp, long endTimestamp) {
diff --cc server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
index e9e9c33,c933dba..389cbde
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
@@@ -94,15 -96,6 +96,14 @@@ public interface IMemTable 
        List<TimeRange> deletionList)
        throws IOException, QueryProcessException, MetadataException;
  
- 
 +  ReadOnlyMemChunk query(
 +      String deviceId,
 +      String measurement,
 +      IMeasurementSchema schema,
 +      long timeLowerBound,
 +      List<TimeRange> deletionList)
 +      throws IOException, QueryProcessException, MetadataException;
 +
    /** putBack all the memory resources. */
    void clear();
  
diff --cc server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
index a3fc963,b3dfb98..60a6de5
--- a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/QueryDataSource.java
@@@ -19,22 -19,27 +19,21 @@@
  
  package org.apache.iotdb.db.engine.querycontext;
  
- import java.util.List;
  import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 -import org.apache.iotdb.db.metadata.PartialPath;
  import org.apache.iotdb.tsfile.read.filter.TimeFilter;
  import org.apache.iotdb.tsfile.read.filter.basic.Filter;
  import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
  
+ import java.util.List;
+ 
  public class QueryDataSource {
 -  private PartialPath seriesPath;
    private List<TsFileResource> seqResources;
    private List<TsFileResource> unseqResources;
  
    /** data older than currentTime - dataTTL should be ignored. */
    private long dataTTL = Long.MAX_VALUE;
  
--  public QueryDataSource(
 -      PartialPath seriesPath,
--      List<TsFileResource> seqResources,
--      List<TsFileResource> unseqResources) {
 -    this.seriesPath = seriesPath;
++  public QueryDataSource(List<TsFileResource> seqResources, List<TsFileResource> unseqResources) {
      this.seqResources = seqResources;
      this.unseqResources = unseqResources;
    }
diff --cc server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index e67b691,1f9d80e..db185fd
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@@ -1635,12 -1640,14 +1637,7 @@@ public class StorageGroupProcessor 
          } else {
            tsFileResource
                .getUnsealedFileProcessor()
--              .query(
-                   deviceId,
-                   fullPath.getMeasurement(),
-                   schema,
 -                  deviceId.getFullPath(),
 -                  measurementId,
 -                  schema.getType(),
 -                  schema.getEncodingType(),
 -                  schema.getProps(),
--                  context,
--                  tsfileResourcesForQuery);
++              .query(deviceId, fullPath.getMeasurement(), schema, context, tsfileResourcesForQuery);
          }
        } catch (IOException e) {
          throw new MetadataException(e);
diff --cc server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 773e5dd,1c0ff40..376779d
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@@ -73,12 -61,25 +63,24 @@@ import org.apache.iotdb.tsfile.file.met
  import org.apache.iotdb.tsfile.read.common.TimeRange;
  import org.apache.iotdb.tsfile.utils.Binary;
  import org.apache.iotdb.tsfile.utils.Pair;
 +import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
  import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
  import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
+ 
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import java.io.File;
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Iterator;
+ import java.util.List;
 -import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentLinkedDeque;
+ import java.util.concurrent.locks.ReadWriteLock;
+ import java.util.concurrent.locks.ReentrantReadWriteLock;
+ 
  @SuppressWarnings("java:S1135") // ignore todos
  public class TsFileProcessor {
  
@@@ -1051,8 -1107,10 +1108,8 @@@
     * memtables and then compact them into one TimeValuePairSorter). Then get the related
     * ChunkMetadata of data on disk.
     *
-    * @param deviceId      device id
+    * @param deviceId device id
     * @param measurementId measurements id
 -   * @param dataType data type
 -   * @param encoding encoding
     */
    @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
    public void query(
@@@ -1080,11 -1140,13 +1137,7 @@@
                  flushingMemTable, deviceId, measurementId, context.getQueryTimeLowerBound());
          ReadOnlyMemChunk memChunk =
              flushingMemTable.query(
--                deviceId,
--                measurementId,
-                 schema,
 -                dataType,
 -                encoding,
 -                props,
--                context.getQueryTimeLowerBound(),
--                deletionList);
++                deviceId, measurementId, schema, context.getQueryTimeLowerBound(), deletionList);
          if (memChunk != null) {
            readOnlyMemChunks.add(memChunk);
          }
@@@ -1092,11 -1154,13 +1145,7 @@@
        if (workMemTable != null) {
          ReadOnlyMemChunk memChunk =
              workMemTable.query(
--                deviceId,
--                measurementId,
-                 schema,
 -                dataType,
 -                encoding,
 -                props,
--                context.getQueryTimeLowerBound(),
--                null);
++                deviceId, measurementId, schema, context.getQueryTimeLowerBound(), null);
          if (memChunk != null) {
            readOnlyMemChunks.add(memChunk);
          }
@@@ -1108,33 -1172,8 +1157,33 @@@
                modificationFile,
                new PartialPath(deviceId + IoTDBConstant.PATH_SEPARATOR + measurementId));
  
 -      List<ChunkMetadata> chunkMetadataList =
 -          writer.getVisibleMetadataList(deviceId, measurementId, dataType);
 +      List<IChunkMetadata> chunkMetadataList = new ArrayList<>();
 +      if (schema instanceof VectorMeasurementSchema) {
 +        List<ChunkMetadata> timeChunkMetadataList =
 +            writer.getVisibleMetadataList(deviceId, measurementId, schema.getType());
 +        List<List<ChunkMetadata>> valueChunkMetadataList = new ArrayList<>();
 +        List<String> valueMeasurementIdList = schema.getValueMeasurementIdList();
 +        List<TSDataType> valueDataTypeList = schema.getValueTSDataTypeList();
 +        for (int i = 0; i < valueMeasurementIdList.size(); i++) {
-           valueChunkMetadataList.add(writer
-               .getVisibleMetadataList(deviceId, valueMeasurementIdList.get(i),
-                   valueDataTypeList.get(i)));
++          valueChunkMetadataList.add(
++              writer.getVisibleMetadataList(
++                  deviceId, valueMeasurementIdList.get(i), valueDataTypeList.get(i)));
 +        }
 +
- 
 +        for (int i = 0; i < timeChunkMetadataList.size(); i++) {
 +          List<IChunkMetadata> valueChunkMetadata = new ArrayList<>();
 +          for (List<ChunkMetadata> chunkMetadata : valueChunkMetadataList) {
 +            valueChunkMetadata.add(chunkMetadata.get(i));
 +          }
-           chunkMetadataList
-               .add(new VectorChunkMetadata(timeChunkMetadataList.get(i), valueChunkMetadata));
++          chunkMetadataList.add(
++              new VectorChunkMetadata(timeChunkMetadataList.get(i), valueChunkMetadata));
 +        }
 +      } else {
 +        chunkMetadataList =
-             new ArrayList<>(writer.getVisibleMetadataList(deviceId, measurementId, schema.getType()));
++            new ArrayList<>(
++                writer.getVisibleMetadataList(deviceId, measurementId, schema.getType()));
 +      }
 +
        QueryUtils.modifyChunkMetaData(chunkMetadataList, modifications);
        chunkMetadataList.removeIf(context::chunkNotSatisfy);
  
diff --cc server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index f6c6974,3fc3160..d9f19cc
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@@ -1013,17 -1093,6 +1093,16 @@@ public class MManager 
      return null;
    }
  
 +  // TODO BY ZESONG SUN
-   public IMeasurementSchema getSeriesSchema(PartialPath fullPath)
-       throws MetadataException {
++  public IMeasurementSchema getSeriesSchema(PartialPath fullPath) throws MetadataException {
 +    MNode node = mtree.getNodeByPath(fullPath.getDevicePath());
 +    MNode leaf = node.getChild(fullPath.getMeasurement());
 +    if (leaf != null) {
 +      return ((MeasurementMNode) leaf).getSchema();
 +    }
 +    return null;
 +  }
 +
    /**
     * Get child node path in the next level of the given path.
     *
diff --cc server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index 0c53d08,3febfa9..a8c2198
--- a/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@@ -19,6 -19,11 +19,11 @@@
  
  package org.apache.iotdb.db.query.context;
  
+ import org.apache.iotdb.db.engine.modification.Modification;
+ import org.apache.iotdb.db.engine.modification.ModificationFile;
+ import org.apache.iotdb.db.metadata.PartialPath;
 -import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
++import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata;
+ 
  import java.util.ArrayList;
  import java.util.HashMap;
  import java.util.List;
diff --cc server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
index 6b64b13,e68c4af..4767a36
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
@@@ -126,7 -125,9 +126,12 @@@ public class StorageGroupProcessorTest 
        tsfileProcessor.query(
            deviceId,
            measurementId,
-           new MeasurementSchema(measurementId, TSDataType.INT32, TSEncoding.RLE, CompressionType.UNCOMPRESSED, Collections.emptyMap()),
 -          TSDataType.INT32,
 -          TSEncoding.RLE,
 -          Collections.emptyMap(),
++          new MeasurementSchema(
++              measurementId,
++              TSDataType.INT32,
++              TSEncoding.RLE,
++              CompressionType.UNCOMPRESSED,
++              Collections.emptyMap()),
            new QueryContext(),
            tsfileResourcesForQuery);
      }
diff --cc server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index 276cacd,f972a83..5f43331
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@@ -46,8 -36,8 +37,9 @@@ import org.apache.iotdb.tsfile.read.Tim
  import org.apache.iotdb.tsfile.read.reader.IPointReader;
  import org.apache.iotdb.tsfile.write.record.TSRecord;
  import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
 +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
  import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
+ 
  import org.junit.After;
  import org.junit.Before;
  import org.junit.Test;
@@@ -104,7 -104,7 +106,12 @@@ public class TsFileProcessorTest 
      SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
      List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertTrue(tsfileResourcesForQuery.isEmpty());
  
      for (int i = 1; i <= 100; i++) {
@@@ -116,7 -116,7 +123,12 @@@
      // query data in memory
      tsfileResourcesForQuery.clear();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
      List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
      for (ReadOnlyMemChunk chunk : memChunks) {
@@@ -134,7 -134,7 +146,12 @@@
  
      tsfileResourcesForQuery.clear();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
      assertEquals(1, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
      assertEquals(
@@@ -166,7 -166,7 +183,12 @@@
      SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
      List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertTrue(tsfileResourcesForQuery.isEmpty());
  
      for (int i = 1; i <= 100; i++) {
@@@ -178,7 -178,7 +200,12 @@@
      // query data in memory
      tsfileResourcesForQuery.clear();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
      int num = 1;
      List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
@@@ -197,7 -197,7 +224,12 @@@
  
      tsfileResourcesForQuery.clear();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
      assertEquals(1, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
      assertEquals(
@@@ -254,7 -254,7 +286,12 @@@
      SystemInfo.getInstance().reportStorageGroupStatus(sgInfo);
      List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertTrue(tsfileResourcesForQuery.isEmpty());
  
      for (int flushId = 0; flushId < 10; flushId++) {
@@@ -269,7 -269,7 +306,12 @@@
  
      tsfileResourcesForQuery.clear();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertFalse(tsfileResourcesForQuery.isEmpty());
      assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
      assertEquals(10, tsfileResourcesForQuery.get(0).getChunkMetadataList().size());
@@@ -302,7 -302,7 +344,12 @@@
      List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
  
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertTrue(tsfileResourcesForQuery.isEmpty());
  
      for (int i = 1; i <= 100; i++) {
@@@ -314,7 -314,7 +361,12 @@@
      // query data in memory
      tsfileResourcesForQuery.clear();
      processor.query(
-         deviceId, measurementId, new MeasurementSchema(measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props), context, tsfileResourcesForQuery);
 -        deviceId, measurementId, dataType, encoding, props, context, tsfileResourcesForQuery);
++        deviceId,
++        measurementId,
++        new MeasurementSchema(
++            measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props),
++        context,
++        tsfileResourcesForQuery);
      assertFalse(tsfileResourcesForQuery.isEmpty());
      assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk().isEmpty());
      List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk();
diff --cc server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
index a713157,55eea5c..a77d592
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReaderTest.java
@@@ -46,6 -38,14 +38,16 @@@ import org.junit.After
  import org.junit.Before;
  import org.junit.Test;
  
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ 
 -import static org.junit.Assert.*;
++import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertTrue;
++import static org.junit.Assert.fail;
+ 
  public class SeriesAggregateReaderTest {
  
    private static final String SERIES_READER_TEST_SG = "root.seriesReaderTest";
diff --cc server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
index 129f18d,4e71afa..7834017
--- a/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestampTest.java
@@@ -61,10 -61,11 +61,7 @@@ public class SeriesReaderByTimestampTes
  
    @Test
    public void test() throws IOException, IllegalPathException {
--    QueryDataSource dataSource =
--        new QueryDataSource(
 -            new PartialPath(SERIES_READER_TEST_SG + ".device0.sensor0"),
--            seqResources,
--            unseqResources);
++    QueryDataSource dataSource = new QueryDataSource(seqResources, unseqResources);
  
      Set<String> allSensors = new HashSet<>();
      allSensors.add("sensor0");


[iotdb] 01/05: add VectorSeriesReader

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6cd892d4722d18ea36a9880877cb92b2a0fe1071
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Fri Mar 12 21:18:40 2021 +0800

    add VectorSeriesReader
---
 .../cluster/query/reader/ClusterReaderFactory.java |  26 ++--
 .../query/reader/series/SeriesAggregateReader.java |  14 +--
 .../reader/series/SeriesRawDataBatchReader.java    |  28 +----
 .../iotdb/db/query/reader/series/SeriesReader.java | 136 ++++++++++-----------
 .../reader/series/SeriesReaderByTimestamp.java     |  19 +--
 .../query/reader/series/SeriesReaderFactory.java   |  53 ++++++++
 .../db/query/reader/series/VectorSeriesReader.java | 115 +++++++++++++++++
 7 files changed, 255 insertions(+), 136 deletions(-)

diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
index 91819b1..2d29033 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
@@ -55,6 +55,7 @@ import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
 import org.apache.iotdb.db.query.reader.series.SeriesRawDataPointReader;
 import org.apache.iotdb.db.query.reader.series.SeriesReader;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.series.SeriesReaderFactory;
 import org.apache.iotdb.db.utils.SerializeUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -207,7 +208,7 @@ public class ClusterReaderFactory {
    * each group a series reader will be created, and finally all such readers will be merged into
    * one.
    *
-   * @param timeFilter nullable, when null, all data groups will be queried
+   * @param timeFilter  nullable, when null, all data groups will be queried
    * @param valueFilter nullable
    */
   public ManagedSeriesReader getSeriesReader(
@@ -253,7 +254,7 @@ public class ClusterReaderFactory {
    * "partitionGroup" contains the local node, a local reader will be returned. Otherwise a remote
    * reader will be returned.
    *
-   * @param timeFilter nullable
+   * @param timeFilter  nullable
    * @param valueFilter nullable
    */
   private IPointReader getSeriesReader(
@@ -313,7 +314,7 @@ public class ClusterReaderFactory {
    *
    * @param path
    * @param dataType
-   * @param timeFilter nullable
+   * @param timeFilter  nullable
    * @param valueFilter nullable
    * @param context
    * @return
@@ -353,7 +354,7 @@ public class ClusterReaderFactory {
    *
    * @param path
    * @param dataType
-   * @param timeFilter nullable
+   * @param timeFilter  nullable
    * @param valueFilter nullable
    * @param context
    * @return
@@ -374,16 +375,9 @@ public class ClusterReaderFactory {
         ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
     QueryDataSource queryDataSource =
         QueryResourceManager.getInstance().getQueryDataSource(path, context, timeFilter);
-    return new SeriesReader(
-        path,
-        allSensors,
-        dataType,
-        context,
-        queryDataSource,
-        timeFilter,
-        valueFilter,
-        new SlotTsFileFilter(nodeSlots),
-        ascending);
+    return SeriesReaderFactory
+        .createSeriesReader(path, allSensors, dataType, context, queryDataSource, timeFilter,
+            valueFilter, new SlotTsFileFilter(nodeSlots), ascending);
   }
 
   /**
@@ -392,7 +386,7 @@ public class ClusterReaderFactory {
    * the id of the reader will be returned so that we can fetch data from that node using the reader
    * id.
    *
-   * @param timeFilter nullable
+   * @param timeFilter  nullable
    * @param valueFilter nullable
    */
   private IPointReader getRemoteSeriesPointReader(
@@ -671,7 +665,7 @@ public class ClusterReaderFactory {
    *
    * @param path
    * @param dataType
-   * @param timeFilter nullable
+   * @param timeFilter  nullable
    * @param valueFilter nullable
    * @param context
    * @return an IBatchReader or null if there is no satisfying data
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
index 04cb907..d87aa26 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
@@ -44,17 +44,9 @@ public class SeriesAggregateReader implements IAggregateReader {
       Filter valueFilter,
       TsFileFilter fileFilter,
       boolean ascending) {
-    this.seriesReader =
-        new SeriesReader(
-            seriesPath,
-            allSensors,
-            dataType,
-            context,
-            dataSource,
-            timeFilter,
-            valueFilter,
-            fileFilter,
-            ascending);
+    this.seriesReader = SeriesReaderFactory
+        .createSeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter,
+            valueFilter, fileFilter, ascending);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
index 8698955..7be1fb2 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
@@ -57,17 +57,9 @@ public class SeriesRawDataBatchReader implements ManagedSeriesReader {
       Filter valueFilter,
       TsFileFilter fileFilter,
       boolean ascending) {
-    this.seriesReader =
-        new SeriesReader(
-            seriesPath,
-            allSensors,
-            dataType,
-            context,
-            dataSource,
-            timeFilter,
-            valueFilter,
-            fileFilter,
-            ascending);
+    this.seriesReader = SeriesReaderFactory
+        .createSeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter,
+            valueFilter, fileFilter, ascending);
   }
 
   @TestOnly
@@ -83,17 +75,9 @@ public class SeriesRawDataBatchReader implements ManagedSeriesReader {
       boolean ascending) {
     Set<String> allSensors = new HashSet<>();
     allSensors.add(seriesPath.getMeasurement());
-    this.seriesReader =
-        new SeriesReader(
-            seriesPath,
-            allSensors,
-            dataType,
-            context,
-            seqFileResource,
-            unseqFileResource,
-            timeFilter,
-            valueFilter,
-            ascending);
+    this.seriesReader = SeriesReaderFactory
+        .createSeriesReader(seriesPath, allSensors, dataType, context, seqFileResource,
+            unseqFileResource, timeFilter, valueFilter, ascending);
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
index 5350993..1051a24 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
@@ -18,6 +18,16 @@
  */
 package org.apache.iotdb.db.query.reader.series;
 
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collectors;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -44,28 +54,17 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.basic.UnaryFilter;
 import org.apache.iotdb.tsfile.read.reader.IPageReader;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Objects;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.function.ToLongFunction;
-import java.util.stream.Collectors;
-
 public class SeriesReader {
 
   // inner class of SeriesReader for order purpose
-  private TimeOrderUtils orderUtils;
+  protected TimeOrderUtils orderUtils;
 
-  private final PartialPath seriesPath;
+  protected final PartialPath seriesPath;
 
   // all the sensors in this device;
-  private final Set<String> allSensors;
-  private final TSDataType dataType;
-  private final QueryContext context;
+  protected final Set<String> allSensors;
+  protected final TSDataType dataType;
+  protected final QueryContext context;
 
   /*
    * There is at most one is not null between timeFilter and valueFilter
@@ -74,44 +73,44 @@ public class SeriesReader {
    *
    * valueFilter is pushed down to non-overlapped page only
    */
-  private final Filter timeFilter;
-  private final Filter valueFilter;
+  protected final Filter timeFilter;
+  protected final Filter valueFilter;
   /*
    * file cache
    */
-  private final List<TsFileResource> seqFileResource;
-  private final List<TsFileResource> unseqFileResource;
+  protected final List<TsFileResource> seqFileResource;
+  protected final List<TsFileResource> unseqFileResource;
 
   /*
    * TimeSeriesMetadata cache
    */
-  private ITimeSeriesMetadata firstTimeSeriesMetadata;
-  private final List<ITimeSeriesMetadata> seqTimeSeriesMetadata = new LinkedList<>();
-  private final PriorityQueue<ITimeSeriesMetadata> unSeqTimeSeriesMetadata;
+  protected ITimeSeriesMetadata firstTimeSeriesMetadata;
+  protected final List<ITimeSeriesMetadata> seqTimeSeriesMetadata = new LinkedList<>();
+  protected final PriorityQueue<ITimeSeriesMetadata> unSeqTimeSeriesMetadata;
 
   /*
    * chunk cache
    */
-  private IChunkMetadata firstChunkMetadata;
-  private final PriorityQueue<IChunkMetadata> cachedChunkMetadata;
+  protected IChunkMetadata firstChunkMetadata;
+  protected final PriorityQueue<IChunkMetadata> cachedChunkMetadata;
 
   /*
    * page cache
    */
-  private VersionPageReader firstPageReader;
-  private final List<VersionPageReader> seqPageReaders = new LinkedList<>();
-  private final PriorityQueue<VersionPageReader> unSeqPageReaders;
+  protected VersionPageReader firstPageReader;
+  protected final List<VersionPageReader> seqPageReaders = new LinkedList<>();
+  protected final PriorityQueue<VersionPageReader> unSeqPageReaders;
 
   /*
    * point cache
    */
-  private final PriorityMergeReader mergeReader;
+  protected final PriorityMergeReader mergeReader;
 
   /*
    * result cache
    */
-  private boolean hasCachedNextOverlappedPage;
-  private BatchData cachedBatchData;
+  protected boolean hasCachedNextOverlappedPage;
+  protected BatchData cachedBatchData;
 
   public SeriesReader(
       PartialPath seriesPath,
@@ -329,7 +328,7 @@ public class SeriesReader {
     }
   }
 
-  private void unpackOneTimeSeriesMetadata(ITimeSeriesMetadata timeSeriesMetadata)
+  protected void unpackOneTimeSeriesMetadata(ITimeSeriesMetadata timeSeriesMetadata)
       throws IOException {
     List<IChunkMetadata> chunkMetadataList =
         FileLoaderUtils.loadChunkMetadataList(timeSeriesMetadata);
@@ -857,36 +856,19 @@ public class SeriesReader {
    * approach is likely to be ubiquitous, but it keeps the system running smoothly
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void tryToUnpackAllOverlappedFilesToTimeSeriesMetadata() throws IOException {
+  protected void tryToUnpackAllOverlappedFilesToTimeSeriesMetadata() throws IOException {
     /*
      * Fill sequence TimeSeriesMetadata List until it is not empty
      */
     while (seqTimeSeriesMetadata.isEmpty() && !seqFileResource.isEmpty()) {
-      TimeseriesMetadata timeseriesMetadata =
-          FileLoaderUtils.loadTimeSeriesMetadata(
-              orderUtils.getNextSeqFileResource(seqFileResource, true),
-              seriesPath,
-              context,
-              getAnyFilter(),
-              allSensors);
-      if (timeseriesMetadata != null) {
-        timeseriesMetadata.setSeq(true);
-        seqTimeSeriesMetadata.add(timeseriesMetadata);
-      }
+      unpackSeqTsFileResource();
     }
 
     /*
      * Fill unSequence TimeSeriesMetadata Priority Queue until it is not empty
      */
     while (unSeqTimeSeriesMetadata.isEmpty() && !unseqFileResource.isEmpty()) {
-      TimeseriesMetadata timeseriesMetadata =
-          FileLoaderUtils.loadTimeSeriesMetadata(
-              unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
-      if (timeseriesMetadata != null) {
-        timeseriesMetadata.setModified(true);
-        timeseriesMetadata.setSeq(false);
-        unSeqTimeSeriesMetadata.add(timeseriesMetadata);
-      }
+      unpackUnseqTsFileResource();
     }
 
     /*
@@ -935,37 +917,45 @@ public class SeriesReader {
     }
   }
 
-  private void unpackAllOverlappedTsFilesToTimeSeriesMetadata(long endpointTime)
+  protected void unpackAllOverlappedTsFilesToTimeSeriesMetadata(long endpointTime)
       throws IOException {
     while (!unseqFileResource.isEmpty()
         && orderUtils.isOverlapped(endpointTime, unseqFileResource.get(0))) {
-      TimeseriesMetadata timeseriesMetadata =
-          FileLoaderUtils.loadTimeSeriesMetadata(
-              unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
-      if (timeseriesMetadata != null) {
-        timeseriesMetadata.setModified(true);
-        timeseriesMetadata.setSeq(false);
-        unSeqTimeSeriesMetadata.add(timeseriesMetadata);
-      }
+      unpackUnseqTsFileResource();
     }
     while (!seqFileResource.isEmpty()
         && orderUtils.isOverlapped(
             endpointTime, orderUtils.getNextSeqFileResource(seqFileResource, false))) {
-      TimeseriesMetadata timeseriesMetadata =
-          FileLoaderUtils.loadTimeSeriesMetadata(
-              orderUtils.getNextSeqFileResource(seqFileResource, true),
-              seriesPath,
-              context,
-              getAnyFilter(),
-              allSensors);
-      if (timeseriesMetadata != null) {
-        timeseriesMetadata.setSeq(true);
-        seqTimeSeriesMetadata.add(timeseriesMetadata);
-      }
+      unpackSeqTsFileResource();
+    }
+  }
+
+  protected void unpackSeqTsFileResource() throws IOException {
+    TimeseriesMetadata timeseriesMetadata =
+        FileLoaderUtils.loadTimeSeriesMetadata(
+            orderUtils.getNextSeqFileResource(seqFileResource, true),
+            seriesPath,
+            context,
+            getAnyFilter(),
+            allSensors);
+    if (timeseriesMetadata != null) {
+      timeseriesMetadata.setSeq(true);
+      seqTimeSeriesMetadata.add(timeseriesMetadata);
+    }
+  }
+
+  protected void unpackUnseqTsFileResource() throws IOException {
+    TimeseriesMetadata timeseriesMetadata =
+        FileLoaderUtils.loadTimeSeriesMetadata(
+            unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
+    if (timeseriesMetadata != null) {
+      timeseriesMetadata.setModified(true);
+      timeseriesMetadata.setSeq(false);
+      unSeqTimeSeriesMetadata.add(timeseriesMetadata);
     }
   }
 
-  private Filter getAnyFilter() {
+  protected Filter getAnyFilter() {
     return timeFilter != null ? timeFilter : valueFilter;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
index f09b980..2063e71 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.query.reader.series;
 
+import java.io.IOException;
+import java.util.Set;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.query.context.QueryContext;
@@ -28,9 +30,6 @@ import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.UnaryFilter;
 
-import java.io.IOException;
-import java.util.Set;
-
 public class SeriesReaderByTimestamp implements IReaderByTimestamp {
 
   private SeriesReader seriesReader;
@@ -47,17 +46,9 @@ public class SeriesReaderByTimestamp implements IReaderByTimestamp {
       boolean ascending) {
     UnaryFilter timeFilter =
         ascending ? TimeFilter.gtEq(Long.MIN_VALUE) : TimeFilter.ltEq(Long.MAX_VALUE);
-    seriesReader =
-        new SeriesReader(
-            seriesPath,
-            allSensors,
-            dataType,
-            context,
-            dataSource,
-            timeFilter,
-            null,
-            fileFilter,
-            ascending);
+    this.seriesReader = SeriesReaderFactory
+        .createSeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter, null,
+            fileFilter, ascending);
     this.ascending = ascending;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
new file mode 100644
index 0000000..562cb2d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.reader.series;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+public class SeriesReaderFactory {
+
+  public static SeriesReader createSeriesReader(PartialPath seriesPath, Set<String> allSensors,
+      TSDataType dataType, QueryContext context, QueryDataSource dataSource, Filter timeFilter,
+      Filter valueFilter, TsFileFilter fileFilter, boolean ascending) {
+    if (seriesPath instanceof VectorPartialPath) {
+      return new VectorSeriesReader(seriesPath, allSensors, dataType, context, dataSource,
+          timeFilter, valueFilter, fileFilter, ascending);
+    } else {
+      return new SeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter,
+          valueFilter, fileFilter, ascending);
+    }
+  }
+
+  public static SeriesReader createSeriesReader(PartialPath seriesPath, Set<String> allSensors,
+      TSDataType dataType, QueryContext context, List<TsFileResource> seqFileResource,
+      List<TsFileResource> unseqFileResource, Filter timeFilter, Filter valueFilter,
+      boolean ascending) {
+    return new SeriesReader(seriesPath, allSensors, dataType, context, seqFileResource,
+        unseqFileResource, timeFilter, valueFilter, ascending);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
new file mode 100644
index 0000000..29e7d13
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
@@ -0,0 +1,115 @@
+/*
+ * 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.reader.series;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.VectorPartialPath;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
+import org.apache.iotdb.db.utils.FileLoaderUtils;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.VectorTimeSeriesMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+public class VectorSeriesReader extends SeriesReader {
+
+  private final VectorPartialPath vectorPartialPath;
+
+  public VectorSeriesReader(PartialPath seriesPath,
+      Set<String> allSensors,
+      TSDataType dataType,
+      QueryContext context,
+      QueryDataSource dataSource,
+      Filter timeFilter,
+      Filter valueFilter,
+      TsFileFilter fileFilter, boolean ascending) {
+    super(seriesPath, allSensors, dataType, context, dataSource, timeFilter, valueFilter,
+        fileFilter,
+        ascending);
+    this.vectorPartialPath = (VectorPartialPath) seriesPath;
+  }
+
+  @TestOnly
+  VectorSeriesReader(PartialPath seriesPath, Set<String> allSensors,
+      TSDataType dataType, QueryContext context,
+      List<TsFileResource> seqFileResource,
+      List<TsFileResource> unseqFileResource,
+      Filter timeFilter, Filter valueFilter, boolean ascending) {
+    super(seriesPath, allSensors, dataType, context, seqFileResource, unseqFileResource, timeFilter,
+        valueFilter, ascending);
+    this.vectorPartialPath = (VectorPartialPath) seriesPath;
+  }
+
+  protected void unpackSeqTsFileResource() throws IOException {
+    TsFileResource resource = orderUtils.getNextSeqFileResource(seqFileResource, true);
+    TimeseriesMetadata timeseriesMetadata = FileLoaderUtils
+        .loadTimeSeriesMetadata(resource, vectorPartialPath, context, getAnyFilter(), allSensors);
+    if (timeseriesMetadata != null) {
+      timeseriesMetadata.setSeq(true);
+      List<TimeseriesMetadata> valueTimeseriesMetadataList = new ArrayList<>();
+      for (PartialPath subSensor : vectorPartialPath.getSubSensorsPathList()) {
+        TimeseriesMetadata valueTimeSeriesMetadata = FileLoaderUtils
+            .loadTimeSeriesMetadata(resource, subSensor, context, getAnyFilter(),
+                Collections.emptySet());
+        if (valueTimeSeriesMetadata == null) {
+          throw new IOException("File contains value");
+        }
+        valueTimeSeriesMetadata.setSeq(true);
+        valueTimeseriesMetadataList.add(valueTimeSeriesMetadata);
+      }
+      VectorTimeSeriesMetadata vectorTimeSeriesMetadata = new VectorTimeSeriesMetadata(
+          timeseriesMetadata, valueTimeseriesMetadataList);
+      seqTimeSeriesMetadata.add(vectorTimeSeriesMetadata);
+    }
+  }
+
+  protected void unpackUnseqTsFileResource() throws IOException {
+    TsFileResource resource = unseqFileResource.remove(0);
+    TimeseriesMetadata timeseriesMetadata = FileLoaderUtils
+        .loadTimeSeriesMetadata(resource, vectorPartialPath, context, getAnyFilter(), allSensors);
+    if (timeseriesMetadata != null) {
+      timeseriesMetadata.setModified(true);
+      timeseriesMetadata.setSeq(false);
+      List<TimeseriesMetadata> valueTimeseriesMetadataList = new ArrayList<>();
+      for (PartialPath subSensor : vectorPartialPath.getSubSensorsPathList()) {
+        TimeseriesMetadata valueTimeSeriesMetadata = FileLoaderUtils
+            .loadTimeSeriesMetadata(resource, subSensor, context, getAnyFilter(),
+                Collections.emptySet());
+        if (valueTimeSeriesMetadata == null) {
+          throw new IOException("File contains value");
+        }
+        timeseriesMetadata.setModified(true);
+        valueTimeSeriesMetadata.setSeq(false);
+        valueTimeseriesMetadataList.add(valueTimeSeriesMetadata);
+      }
+      VectorTimeSeriesMetadata vectorTimeSeriesMetadata = new VectorTimeSeriesMetadata(
+          timeseriesMetadata, valueTimeseriesMetadataList);
+      unSeqTimeSeriesMetadata.add(vectorTimeSeriesMetadata);
+    }
+  }
+}


[iotdb] 02/05: format code

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b8b94774434ae62a11b79e2532ab12394ed67ab5
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Fri Mar 12 21:19:18 2021 +0800

    format code
---
 .../cluster/query/reader/ClusterReaderFactory.java | 25 ++++---
 .../query/reader/series/SeriesAggregateReader.java | 14 +++-
 .../reader/series/SeriesRawDataBatchReader.java    | 28 ++++++--
 .../iotdb/db/query/reader/series/SeriesReader.java | 21 +++---
 .../reader/series/SeriesReaderByTimestamp.java     | 19 ++++--
 .../query/reader/series/SeriesReaderFactory.java   | 66 ++++++++++++++----
 .../db/query/reader/series/VectorSeriesReader.java | 79 ++++++++++++++--------
 7 files changed, 178 insertions(+), 74 deletions(-)

diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
index 2d29033..1ddf5c8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/ClusterReaderFactory.java
@@ -208,7 +208,7 @@ public class ClusterReaderFactory {
    * each group a series reader will be created, and finally all such readers will be merged into
    * one.
    *
-   * @param timeFilter  nullable, when null, all data groups will be queried
+   * @param timeFilter nullable, when null, all data groups will be queried
    * @param valueFilter nullable
    */
   public ManagedSeriesReader getSeriesReader(
@@ -254,7 +254,7 @@ public class ClusterReaderFactory {
    * "partitionGroup" contains the local node, a local reader will be returned. Otherwise a remote
    * reader will be returned.
    *
-   * @param timeFilter  nullable
+   * @param timeFilter nullable
    * @param valueFilter nullable
    */
   private IPointReader getSeriesReader(
@@ -314,7 +314,7 @@ public class ClusterReaderFactory {
    *
    * @param path
    * @param dataType
-   * @param timeFilter  nullable
+   * @param timeFilter nullable
    * @param valueFilter nullable
    * @param context
    * @return
@@ -354,7 +354,7 @@ public class ClusterReaderFactory {
    *
    * @param path
    * @param dataType
-   * @param timeFilter  nullable
+   * @param timeFilter nullable
    * @param valueFilter nullable
    * @param context
    * @return
@@ -375,9 +375,16 @@ public class ClusterReaderFactory {
         ((SlotPartitionTable) metaGroupMember.getPartitionTable()).getNodeSlots(header);
     QueryDataSource queryDataSource =
         QueryResourceManager.getInstance().getQueryDataSource(path, context, timeFilter);
-    return SeriesReaderFactory
-        .createSeriesReader(path, allSensors, dataType, context, queryDataSource, timeFilter,
-            valueFilter, new SlotTsFileFilter(nodeSlots), ascending);
+    return SeriesReaderFactory.createSeriesReader(
+        path,
+        allSensors,
+        dataType,
+        context,
+        queryDataSource,
+        timeFilter,
+        valueFilter,
+        new SlotTsFileFilter(nodeSlots),
+        ascending);
   }
 
   /**
@@ -386,7 +393,7 @@ public class ClusterReaderFactory {
    * the id of the reader will be returned so that we can fetch data from that node using the reader
    * id.
    *
-   * @param timeFilter  nullable
+   * @param timeFilter nullable
    * @param valueFilter nullable
    */
   private IPointReader getRemoteSeriesPointReader(
@@ -665,7 +672,7 @@ public class ClusterReaderFactory {
    *
    * @param path
    * @param dataType
-   * @param timeFilter  nullable
+   * @param timeFilter nullable
    * @param valueFilter nullable
    * @param context
    * @return an IBatchReader or null if there is no satisfying data
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
index d87aa26..a76327d 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesAggregateReader.java
@@ -44,9 +44,17 @@ public class SeriesAggregateReader implements IAggregateReader {
       Filter valueFilter,
       TsFileFilter fileFilter,
       boolean ascending) {
-    this.seriesReader = SeriesReaderFactory
-        .createSeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter,
-            valueFilter, fileFilter, ascending);
+    this.seriesReader =
+        SeriesReaderFactory.createSeriesReader(
+            seriesPath,
+            allSensors,
+            dataType,
+            context,
+            dataSource,
+            timeFilter,
+            valueFilter,
+            fileFilter,
+            ascending);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
index 7be1fb2..d53db04 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesRawDataBatchReader.java
@@ -57,9 +57,17 @@ public class SeriesRawDataBatchReader implements ManagedSeriesReader {
       Filter valueFilter,
       TsFileFilter fileFilter,
       boolean ascending) {
-    this.seriesReader = SeriesReaderFactory
-        .createSeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter,
-            valueFilter, fileFilter, ascending);
+    this.seriesReader =
+        SeriesReaderFactory.createSeriesReader(
+            seriesPath,
+            allSensors,
+            dataType,
+            context,
+            dataSource,
+            timeFilter,
+            valueFilter,
+            fileFilter,
+            ascending);
   }
 
   @TestOnly
@@ -75,9 +83,17 @@ public class SeriesRawDataBatchReader implements ManagedSeriesReader {
       boolean ascending) {
     Set<String> allSensors = new HashSet<>();
     allSensors.add(seriesPath.getMeasurement());
-    this.seriesReader = SeriesReaderFactory
-        .createSeriesReader(seriesPath, allSensors, dataType, context, seqFileResource,
-            unseqFileResource, timeFilter, valueFilter, ascending);
+    this.seriesReader =
+        SeriesReaderFactory.createSeriesReader(
+            seriesPath,
+            allSensors,
+            dataType,
+            context,
+            seqFileResource,
+            unseqFileResource,
+            timeFilter,
+            valueFilter,
+            ascending);
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
index 1051a24..080e2ab 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
@@ -18,16 +18,6 @@
  */
 package org.apache.iotdb.db.query.reader.series;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Objects;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.function.ToLongFunction;
-import java.util.stream.Collectors;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
@@ -54,6 +44,17 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.basic.UnaryFilter;
 import org.apache.iotdb.tsfile.read.reader.IPageReader;
 
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collectors;
+
 public class SeriesReader {
 
   // inner class of SeriesReader for order purpose
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
index 2063e71..053ffa3 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
@@ -18,8 +18,6 @@
  */
 package org.apache.iotdb.db.query.reader.series;
 
-import java.io.IOException;
-import java.util.Set;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.query.context.QueryContext;
@@ -30,6 +28,9 @@ import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.UnaryFilter;
 
+import java.io.IOException;
+import java.util.Set;
+
 public class SeriesReaderByTimestamp implements IReaderByTimestamp {
 
   private SeriesReader seriesReader;
@@ -46,9 +47,17 @@ public class SeriesReaderByTimestamp implements IReaderByTimestamp {
       boolean ascending) {
     UnaryFilter timeFilter =
         ascending ? TimeFilter.gtEq(Long.MIN_VALUE) : TimeFilter.ltEq(Long.MAX_VALUE);
-    this.seriesReader = SeriesReaderFactory
-        .createSeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter, null,
-            fileFilter, ascending);
+    this.seriesReader =
+        SeriesReaderFactory.createSeriesReader(
+            seriesPath,
+            allSensors,
+            dataType,
+            context,
+            dataSource,
+            timeFilter,
+            null,
+            fileFilter,
+            ascending);
     this.ascending = ascending;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
index 562cb2d..f0a1a47 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderFactory.java
@@ -18,8 +18,6 @@
  */
 package org.apache.iotdb.db.query.reader.series;
 
-import java.util.List;
-import java.util.Set;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
@@ -29,25 +27,65 @@ import org.apache.iotdb.db.query.filter.TsFileFilter;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
+import java.util.List;
+import java.util.Set;
+
 public class SeriesReaderFactory {
 
-  public static SeriesReader createSeriesReader(PartialPath seriesPath, Set<String> allSensors,
-      TSDataType dataType, QueryContext context, QueryDataSource dataSource, Filter timeFilter,
-      Filter valueFilter, TsFileFilter fileFilter, boolean ascending) {
+  public static SeriesReader createSeriesReader(
+      PartialPath seriesPath,
+      Set<String> allSensors,
+      TSDataType dataType,
+      QueryContext context,
+      QueryDataSource dataSource,
+      Filter timeFilter,
+      Filter valueFilter,
+      TsFileFilter fileFilter,
+      boolean ascending) {
     if (seriesPath instanceof VectorPartialPath) {
-      return new VectorSeriesReader(seriesPath, allSensors, dataType, context, dataSource,
-          timeFilter, valueFilter, fileFilter, ascending);
+      return new VectorSeriesReader(
+          seriesPath,
+          allSensors,
+          dataType,
+          context,
+          dataSource,
+          timeFilter,
+          valueFilter,
+          fileFilter,
+          ascending);
     } else {
-      return new SeriesReader(seriesPath, allSensors, dataType, context, dataSource, timeFilter,
-          valueFilter, fileFilter, ascending);
+      return new SeriesReader(
+          seriesPath,
+          allSensors,
+          dataType,
+          context,
+          dataSource,
+          timeFilter,
+          valueFilter,
+          fileFilter,
+          ascending);
     }
   }
 
-  public static SeriesReader createSeriesReader(PartialPath seriesPath, Set<String> allSensors,
-      TSDataType dataType, QueryContext context, List<TsFileResource> seqFileResource,
-      List<TsFileResource> unseqFileResource, Filter timeFilter, Filter valueFilter,
+  public static SeriesReader createSeriesReader(
+      PartialPath seriesPath,
+      Set<String> allSensors,
+      TSDataType dataType,
+      QueryContext context,
+      List<TsFileResource> seqFileResource,
+      List<TsFileResource> unseqFileResource,
+      Filter timeFilter,
+      Filter valueFilter,
       boolean ascending) {
-    return new SeriesReader(seriesPath, allSensors, dataType, context, seqFileResource,
-        unseqFileResource, timeFilter, valueFilter, ascending);
+    return new SeriesReader(
+        seriesPath,
+        allSensors,
+        dataType,
+        context,
+        seqFileResource,
+        unseqFileResource,
+        timeFilter,
+        valueFilter,
+        ascending);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
index 29e7d13..265bb5e 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/VectorSeriesReader.java
@@ -18,11 +18,6 @@
  */
 package org.apache.iotdb.db.query.reader.series;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.metadata.PartialPath;
@@ -36,70 +31,100 @@ import org.apache.iotdb.tsfile.file.metadata.VectorTimeSeriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
 public class VectorSeriesReader extends SeriesReader {
 
   private final VectorPartialPath vectorPartialPath;
 
-  public VectorSeriesReader(PartialPath seriesPath,
+  public VectorSeriesReader(
+      PartialPath seriesPath,
       Set<String> allSensors,
       TSDataType dataType,
       QueryContext context,
       QueryDataSource dataSource,
       Filter timeFilter,
       Filter valueFilter,
-      TsFileFilter fileFilter, boolean ascending) {
-    super(seriesPath, allSensors, dataType, context, dataSource, timeFilter, valueFilter,
+      TsFileFilter fileFilter,
+      boolean ascending) {
+    super(
+        seriesPath,
+        allSensors,
+        dataType,
+        context,
+        dataSource,
+        timeFilter,
+        valueFilter,
         fileFilter,
         ascending);
     this.vectorPartialPath = (VectorPartialPath) seriesPath;
   }
 
   @TestOnly
-  VectorSeriesReader(PartialPath seriesPath, Set<String> allSensors,
-      TSDataType dataType, QueryContext context,
+  VectorSeriesReader(
+      PartialPath seriesPath,
+      Set<String> allSensors,
+      TSDataType dataType,
+      QueryContext context,
       List<TsFileResource> seqFileResource,
       List<TsFileResource> unseqFileResource,
-      Filter timeFilter, Filter valueFilter, boolean ascending) {
-    super(seriesPath, allSensors, dataType, context, seqFileResource, unseqFileResource, timeFilter,
-        valueFilter, ascending);
+      Filter timeFilter,
+      Filter valueFilter,
+      boolean ascending) {
+    super(
+        seriesPath,
+        allSensors,
+        dataType,
+        context,
+        seqFileResource,
+        unseqFileResource,
+        timeFilter,
+        valueFilter,
+        ascending);
     this.vectorPartialPath = (VectorPartialPath) seriesPath;
   }
 
   protected void unpackSeqTsFileResource() throws IOException {
     TsFileResource resource = orderUtils.getNextSeqFileResource(seqFileResource, true);
-    TimeseriesMetadata timeseriesMetadata = FileLoaderUtils
-        .loadTimeSeriesMetadata(resource, vectorPartialPath, context, getAnyFilter(), allSensors);
+    TimeseriesMetadata timeseriesMetadata =
+        FileLoaderUtils.loadTimeSeriesMetadata(
+            resource, vectorPartialPath, context, getAnyFilter(), allSensors);
     if (timeseriesMetadata != null) {
       timeseriesMetadata.setSeq(true);
       List<TimeseriesMetadata> valueTimeseriesMetadataList = new ArrayList<>();
       for (PartialPath subSensor : vectorPartialPath.getSubSensorsPathList()) {
-        TimeseriesMetadata valueTimeSeriesMetadata = FileLoaderUtils
-            .loadTimeSeriesMetadata(resource, subSensor, context, getAnyFilter(),
-                Collections.emptySet());
+        TimeseriesMetadata valueTimeSeriesMetadata =
+            FileLoaderUtils.loadTimeSeriesMetadata(
+                resource, subSensor, context, getAnyFilter(), Collections.emptySet());
         if (valueTimeSeriesMetadata == null) {
           throw new IOException("File contains value");
         }
         valueTimeSeriesMetadata.setSeq(true);
         valueTimeseriesMetadataList.add(valueTimeSeriesMetadata);
       }
-      VectorTimeSeriesMetadata vectorTimeSeriesMetadata = new VectorTimeSeriesMetadata(
-          timeseriesMetadata, valueTimeseriesMetadataList);
+      VectorTimeSeriesMetadata vectorTimeSeriesMetadata =
+          new VectorTimeSeriesMetadata(timeseriesMetadata, valueTimeseriesMetadataList);
       seqTimeSeriesMetadata.add(vectorTimeSeriesMetadata);
     }
   }
 
   protected void unpackUnseqTsFileResource() throws IOException {
     TsFileResource resource = unseqFileResource.remove(0);
-    TimeseriesMetadata timeseriesMetadata = FileLoaderUtils
-        .loadTimeSeriesMetadata(resource, vectorPartialPath, context, getAnyFilter(), allSensors);
+    TimeseriesMetadata timeseriesMetadata =
+        FileLoaderUtils.loadTimeSeriesMetadata(
+            resource, vectorPartialPath, context, getAnyFilter(), allSensors);
     if (timeseriesMetadata != null) {
       timeseriesMetadata.setModified(true);
       timeseriesMetadata.setSeq(false);
       List<TimeseriesMetadata> valueTimeseriesMetadataList = new ArrayList<>();
       for (PartialPath subSensor : vectorPartialPath.getSubSensorsPathList()) {
-        TimeseriesMetadata valueTimeSeriesMetadata = FileLoaderUtils
-            .loadTimeSeriesMetadata(resource, subSensor, context, getAnyFilter(),
-                Collections.emptySet());
+        TimeseriesMetadata valueTimeSeriesMetadata =
+            FileLoaderUtils.loadTimeSeriesMetadata(
+                resource, subSensor, context, getAnyFilter(), Collections.emptySet());
         if (valueTimeSeriesMetadata == null) {
           throw new IOException("File contains value");
         }
@@ -107,8 +132,8 @@ public class VectorSeriesReader extends SeriesReader {
         valueTimeSeriesMetadata.setSeq(false);
         valueTimeseriesMetadataList.add(valueTimeSeriesMetadata);
       }
-      VectorTimeSeriesMetadata vectorTimeSeriesMetadata = new VectorTimeSeriesMetadata(
-          timeseriesMetadata, valueTimeseriesMetadataList);
+      VectorTimeSeriesMetadata vectorTimeSeriesMetadata =
+          new VectorTimeSeriesMetadata(timeseriesMetadata, valueTimeseriesMetadataList);
       unSeqTimeSeriesMetadata.add(vectorTimeSeriesMetadata);
     }
   }


[iotdb] 04/05: add todo

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b7fe308816b17cf92f0143eb5211948cecfd725e
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Mon Mar 15 09:13:50 2021 +0800

    add todo
---
 .../main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index 4d68a78..ff9659b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -287,7 +287,7 @@ public abstract class AbstractMemTable implements IMemTable {
     return null;
   }
 
-    @Override
+  @Override
   public void delete(
       PartialPath originalPath, PartialPath devicePath, long startTimestamp, long endTimestamp) {
     Map<String, IWritableMemChunk> deviceMap = memTableMap.get(devicePath.getFullPath());