You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ji...@apache.org on 2020/03/02 02:12:57 UTC

[incubator-iotdb] branch cherry_pick_cluster created (now be0e107)

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

jiangtian pushed a change to branch cherry_pick_cluster
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git.


      at be0e107  cherry pick the changes of the distributed version

This branch includes the following new commits:

     new be0e107  cherry pick the changes of the distributed version

The 1 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.



[incubator-iotdb] 01/01: cherry pick the changes of the distributed version

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

jiangtian pushed a commit to branch cherry_pick_cluster
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit be0e107ebfb8f9806865bed600f218ef61556b6f
Author: jt2594838 <jt...@163.com>
AuthorDate: Mon Mar 2 10:12:43 2020 +0800

    cherry pick the changes of the distributed version
---
 .../resources/conf/iotdb-engine.properties         |   1 +
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 123 +++++++++++----------
 .../ServerConfigConsistent.java}                   |  14 ++-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |  83 ++++++++++++--
 .../engine/storagegroup/StorageGroupProcessor.java |  93 +++++++++-------
 .../db/engine/storagegroup/TsFileResource.java     |  35 +++++-
 .../iotdb/db/exception/StorageEngineException.java |   6 +-
 .../exception/metadata/IllegalPathException.java   |   4 +-
 .../db/exception/metadata/MetadataException.java   |   4 +
 .../exception/metadata/PathNotExistException.java  |   3 -
 .../metadata/StorageGroupAlreadySetException.java  |   3 +-
 .../db/exception/query/QueryProcessException.java  |   2 +-
 .../runtime/StorageEngineFailureException.java     |   4 +-
 .../StorageGroupProcessorException.java            |   2 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |  68 ++++++++++++
 .../java/org/apache/iotdb/db/metadata/MTree.java   |  61 ++++++++++
 .../main/java/org/apache/iotdb/db/qp/Planner.java  |   9 +-
 .../apache/iotdb/db/qp/executor/IPlanExecutor.java |   7 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  67 ++++++-----
 .../iotdb/db/qp/logical/crud/FilterOperator.java   |  10 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  37 ++++++-
 .../iotdb/db/qp/physical/crud/BatchInsertPlan.java |  93 +++++++++++++++-
 .../iotdb/db/qp/physical/crud/DeletePlan.java      |  10 ++
 .../iotdb/db/qp/physical/crud/InsertPlan.java      |  25 +++++
 .../apache/iotdb/db/qp/physical/sys/CountPlan.java |   3 +
 .../db/qp/physical/sys/CreateTimeSeriesPlan.java   |  66 +++++++++--
 .../db/qp/physical/sys/SetStorageGroupPlan.java    |  48 +++++++-
 .../db/qp/physical/sys/ShowChildPathsPlan.java     |   2 +
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  23 +++-
 .../qp/strategy/optimizer/ConcatPathOptimizer.java |  23 +++-
 .../db/query/aggregation/AggregateResult.java      |   5 +
 .../db/query/control/QueryResourceManager.java     |   2 +-
 .../db/query/dataset/AlignByDeviceDataSet.java     |   2 +-
 .../dataset/RawQueryDataSetWithValueFilter.java    |   6 +-
 .../groupby/GroupByWithValueFilterDataSet.java     |   1 +
 .../groupby/GroupByWithoutValueFilterDataSet.java  |   1 +
 .../db/query/executor/AggregationExecutor.java     |  73 ++++++++----
 .../iotdb/db/query/executor/QueryRouter.java       |  12 +-
 .../db/query/executor/RawDataQueryExecutor.java    |  30 +++--
 .../db/query/externalsort/ExternalSortJob.java     |   3 +-
 .../externalsort/ExternalSortJobScheduler.java     |  19 ++--
 .../externalsort/SimpleExternalSortEngine.java     |  18 +--
 .../reader/series/SeriesReaderByTimestamp.java     |   4 +
 .../reader/universal/PriorityMergeReader.java      |   2 +-
 .../query/timegenerator/ServerTimeGenerator.java   |   8 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   4 +-
 .../org/apache/iotdb/db/service/ServiceType.java   |   3 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  79 +++++--------
 .../db/sync/sender/manage/SyncFileManager.java     |   1 +
 .../apache/iotdb/db/tools/IoTDBDataDirViewer.java  |   2 +-
 .../iotdb/db/tools/TsFileResourcePrinter.java      |   2 +-
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |   2 +-
 .../org/apache/iotdb/db/utils/SchemaUtils.java     |  85 +++++++++++++-
 .../java/org/apache/iotdb/db/utils/TestOnly.java   |   4 +
 .../writelog/recover/TsFileRecoverPerformer.java   |   2 +-
 .../apache/iotdb/db/engine/merge/MergeLogTest.java |   3 +-
 .../iotdb/db/engine/merge/MergeOverLapTest.java    |   5 +-
 .../iotdb/db/engine/merge/MergeTaskTest.java       |   5 +-
 .../apache/iotdb/db/engine/merge/MergeTest.java    |   5 +-
 .../engine/modification/DeletionFileNodeTest.java  |   9 +-
 .../db/engine/modification/DeletionQueryTest.java  |   6 +-
 .../storagegroup/FileNodeManagerBenchmark.java     |   7 +-
 .../iotdb/db/engine/storagegroup/TTLTest.java      |   4 +-
 .../iotdb/db/integration/IoTDBMultiSeriesIT.java   |   4 +-
 .../db/integration/IoTDBSequenceDataQueryIT.java   |   5 +-
 .../iotdb/db/integration/IoTDBSeriesReaderIT.java  |   3 +-
 .../apache/iotdb/db/integration/IoTDBTtlIT.java    |   1 -
 .../iotdb/db/qp/plan/ConcatOptimizerTest.java      |   3 +-
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |   6 +-
 .../apache/iotdb/tsfile/read/TimeValuePair.java    |   2 +
 .../apache/iotdb/tsfile/read/common/BatchData.java |  12 +-
 .../iotdb/tsfile/read/filter/GroupByFilter.java    |   8 ++
 .../iotdb/tsfile/read/filter/operator/In.java      |   4 +
 .../apache/iotdb/tsfile/utils/Murmur128Hash.java   |  70 +++++++-----
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |   7 +-
 .../tsfile/write/schema/MeasurementSchema.java     |   1 +
 76 files changed, 1074 insertions(+), 395 deletions(-)

diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index eb4e32d..b429d7c 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -22,6 +22,7 @@
 ####################
 
 metrics_port=8181
+enable_metric_service=false
 
 ####################
 ### RPC Configuration
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 8e8ed0f..deb6c2b 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -47,6 +47,9 @@ public class IoTDBConfig {
    * Port which the metrics service listens to.
    */
   private int metricsPort = 8181;
+
+  private boolean enableMetricService = false;
+
   private String rpcAddress = "0.0.0.0";
 
   /**
@@ -494,7 +497,7 @@ public class IoTDBConfig {
     return memtableNumInEachStorageGroup;
   }
 
-  public void setMemtableNumInEachStorageGroup(int memtableNumInEachStorageGroup) {
+  void setMemtableNumInEachStorageGroup(int memtableNumInEachStorageGroup) {
     this.memtableNumInEachStorageGroup = memtableNumInEachStorageGroup;
   }
 
@@ -610,6 +613,14 @@ public class IoTDBConfig {
     return dataDirs;
   }
 
+  public boolean isEnableMetricService() {
+    return enableMetricService;
+  }
+
+  void setEnableMetricService(boolean enableMetricService) {
+    this.enableMetricService = enableMetricService;
+  }
+
   void setDataDirs(String[] dataDirs) {
     this.dataDirs = dataDirs;
   }
@@ -618,7 +629,7 @@ public class IoTDBConfig {
     return metricsPort;
   }
 
-  public void setMetricsPort(int metricsPort) {
+  void setMetricsPort(int metricsPort) {
     this.metricsPort = metricsPort;
   }
 
@@ -642,7 +653,7 @@ public class IoTDBConfig {
     return timestampPrecision;
   }
 
-  public void setTimestampPrecision(String timestampPrecision) {
+  void setTimestampPrecision(String timestampPrecision) {
     this.timestampPrecision = timestampPrecision;
   }
 
@@ -690,7 +701,7 @@ public class IoTDBConfig {
     return syncDir;
   }
 
-  public void setSyncDir(String syncDir) {
+  void setSyncDir(String syncDir) {
     this.syncDir = syncDir;
   }
 
@@ -698,7 +709,7 @@ public class IoTDBConfig {
     return queryDir;
   }
 
-  public void setQueryDir(String queryDir) {
+  void setQueryDir(String queryDir) {
     this.queryDir = queryDir;
   }
 
@@ -778,7 +789,7 @@ public class IoTDBConfig {
     return rpcMaxConcurrentClientNum;
   }
 
-  public void setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) {
+  void setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) {
     this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum;
   }
 
@@ -822,7 +833,7 @@ public class IoTDBConfig {
     this.syncServerPort = syncServerPort;
   }
 
-  public String getLanguageVersion() {
+  String getLanguageVersion() {
     return languageVersion;
   }
 
@@ -834,7 +845,7 @@ public class IoTDBConfig {
     return baseDir;
   }
 
-  public void setBaseDir(String baseDir) {
+  void setBaseDir(String baseDir) {
     this.baseDir = baseDir;
   }
 
@@ -890,7 +901,7 @@ public class IoTDBConfig {
     return mergeMemoryBudget;
   }
 
-  public void setMergeMemoryBudget(long mergeMemoryBudget) {
+  void setMergeMemoryBudget(long mergeMemoryBudget) {
     this.mergeMemoryBudget = mergeMemoryBudget;
   }
 
@@ -898,7 +909,7 @@ public class IoTDBConfig {
     return mergeThreadNum;
   }
 
-  public void setMergeThreadNum(int mergeThreadNum) {
+  void setMergeThreadNum(int mergeThreadNum) {
     this.mergeThreadNum = mergeThreadNum;
   }
 
@@ -906,7 +917,7 @@ public class IoTDBConfig {
     return continueMergeAfterReboot;
   }
 
-  public void setContinueMergeAfterReboot(boolean continueMergeAfterReboot) {
+  void setContinueMergeAfterReboot(boolean continueMergeAfterReboot) {
     this.continueMergeAfterReboot = continueMergeAfterReboot;
   }
 
@@ -914,7 +925,7 @@ public class IoTDBConfig {
     return mergeIntervalSec;
   }
 
-  public void setMergeIntervalSec(long mergeIntervalSec) {
+  void setMergeIntervalSec(long mergeIntervalSec) {
     this.mergeIntervalSec = mergeIntervalSec;
   }
 
@@ -934,11 +945,11 @@ public class IoTDBConfig {
     this.allocateMemoryForWrite = allocateMemoryForWrite;
   }
 
-  public long getAllocateMemoryForRead() {
+  long getAllocateMemoryForRead() {
     return allocateMemoryForRead;
   }
 
-  public void setAllocateMemoryForRead(long allocateMemoryForRead) {
+  void setAllocateMemoryForRead(long allocateMemoryForRead) {
     this.allocateMemoryForRead = allocateMemoryForRead;
   }
 
@@ -946,7 +957,7 @@ public class IoTDBConfig {
     return enableExternalSort;
   }
 
-  public void setEnableExternalSort(boolean enableExternalSort) {
+  void setEnableExternalSort(boolean enableExternalSort) {
     this.enableExternalSort = enableExternalSort;
   }
 
@@ -954,7 +965,7 @@ public class IoTDBConfig {
     return externalSortThreshold;
   }
 
-  public void setExternalSortThreshold(int externalSortThreshold) {
+  void setExternalSortThreshold(int externalSortThreshold) {
     this.externalSortThreshold = externalSortThreshold;
   }
 
@@ -970,7 +981,7 @@ public class IoTDBConfig {
     return performanceStatDisplayInterval;
   }
 
-  public void setPerformanceStatDisplayInterval(long performanceStatDisplayInterval) {
+  void setPerformanceStatDisplayInterval(long performanceStatDisplayInterval) {
     this.performanceStatDisplayInterval = performanceStatDisplayInterval;
   }
 
@@ -978,7 +989,7 @@ public class IoTDBConfig {
     return performanceStatMemoryInKB;
   }
 
-  public void setPerformanceStatMemoryInKB(int performanceStatMemoryInKB) {
+  void setPerformanceStatMemoryInKB(int performanceStatMemoryInKB) {
     this.performanceStatMemoryInKB = performanceStatMemoryInKB;
   }
 
@@ -986,7 +997,7 @@ public class IoTDBConfig {
     return forceFullMerge;
   }
 
-  public void setForceFullMerge(boolean forceFullMerge) {
+  void setForceFullMerge(boolean forceFullMerge) {
     this.forceFullMerge = forceFullMerge;
   }
 
@@ -1019,7 +1030,7 @@ public class IoTDBConfig {
     return mergeChunkSubThreadNum;
   }
 
-  public void setMergeChunkSubThreadNum(int mergeChunkSubThreadNum) {
+  void setMergeChunkSubThreadNum(int mergeChunkSubThreadNum) {
     this.mergeChunkSubThreadNum = mergeChunkSubThreadNum;
   }
 
@@ -1027,7 +1038,7 @@ public class IoTDBConfig {
     return mergeFileSelectionTimeBudget;
   }
 
-  public void setMergeFileSelectionTimeBudget(long mergeFileSelectionTimeBudget) {
+  void setMergeFileSelectionTimeBudget(long mergeFileSelectionTimeBudget) {
     this.mergeFileSelectionTimeBudget = mergeFileSelectionTimeBudget;
   }
 
@@ -1035,7 +1046,7 @@ public class IoTDBConfig {
     return rpcThriftCompressionEnable;
   }
 
-  public void setRpcThriftCompressionEnable(boolean rpcThriftCompressionEnable) {
+  void setRpcThriftCompressionEnable(boolean rpcThriftCompressionEnable) {
     this.rpcThriftCompressionEnable = rpcThriftCompressionEnable;
   }
 
@@ -1051,7 +1062,7 @@ public class IoTDBConfig {
     return allocateMemoryForFileMetaDataCache;
   }
 
-  public void setAllocateMemoryForFileMetaDataCache(long allocateMemoryForFileMetaDataCache) {
+  void setAllocateMemoryForFileMetaDataCache(long allocateMemoryForFileMetaDataCache) {
     this.allocateMemoryForFileMetaDataCache = allocateMemoryForFileMetaDataCache;
   }
 
@@ -1095,7 +1106,7 @@ public class IoTDBConfig {
     this.watermarkBitString = watermarkBitString;
   }
 
-  public String getWatermarkMethod() {
+  String getWatermarkMethod() {
     return this.watermarkMethod;
   }
 
@@ -1115,7 +1126,7 @@ public class IoTDBConfig {
     return Integer.parseInt(getWatermarkParamValue("embed_lsb_num", "5"));
   }
 
-  public String getWatermarkParamValue(String key, String defaultValue) {
+  private String getWatermarkParamValue(String key, String defaultValue) {
     String res = getWatermarkParamValue(key);
     if (res != null) {
       return res;
@@ -1123,7 +1134,7 @@ public class IoTDBConfig {
     return defaultValue;
   }
 
-  public String getWatermarkParamValue(String key) {
+  private String getWatermarkParamValue(String key) {
     String pattern = key + "=(\\w*)";
     Pattern r = Pattern.compile(pattern);
     Matcher m = r.matcher(watermarkMethod);
@@ -1145,7 +1156,7 @@ public class IoTDBConfig {
     return defaultStorageGroupLevel;
   }
 
-  public void setDefaultStorageGroupLevel(int defaultStorageGroupLevel) {
+  void setDefaultStorageGroupLevel(int defaultStorageGroupLevel) {
     this.defaultStorageGroupLevel = defaultStorageGroupLevel;
   }
 
@@ -1157,7 +1168,7 @@ public class IoTDBConfig {
     this.defaultBooleanEncoding = defaultBooleanEncoding;
   }
 
-  public void setDefaultBooleanEncoding(String defaultBooleanEncoding) {
+  void setDefaultBooleanEncoding(String defaultBooleanEncoding) {
     this.defaultBooleanEncoding = TSEncoding.valueOf(defaultBooleanEncoding);
   }
 
@@ -1169,7 +1180,7 @@ public class IoTDBConfig {
     this.defaultInt32Encoding = defaultInt32Encoding;
   }
 
-  public void setDefaultInt32Encoding(String defaultInt32Encoding) {
+  void setDefaultInt32Encoding(String defaultInt32Encoding) {
     this.defaultInt32Encoding = TSEncoding.valueOf(defaultInt32Encoding);
   }
 
@@ -1181,7 +1192,7 @@ public class IoTDBConfig {
     this.defaultInt64Encoding = defaultInt64Encoding;
   }
 
-  public void setDefaultInt64Encoding(String defaultInt64Encoding) {
+  void setDefaultInt64Encoding(String defaultInt64Encoding) {
     this.defaultInt64Encoding = TSEncoding.valueOf(defaultInt64Encoding);
   }
 
@@ -1193,7 +1204,7 @@ public class IoTDBConfig {
     this.defaultFloatEncoding = defaultFloatEncoding;
   }
 
-  public void setDefaultFloatEncoding(String defaultFloatEncoding) {
+  void setDefaultFloatEncoding(String defaultFloatEncoding) {
     this.defaultFloatEncoding = TSEncoding.valueOf(defaultFloatEncoding);
   }
 
@@ -1205,7 +1216,7 @@ public class IoTDBConfig {
     this.defaultDoubleEncoding = defaultDoubleEncoding;
   }
 
-  public void setDefaultDoubleEncoding(String defaultDoubleEncoding) {
+  void setDefaultDoubleEncoding(String defaultDoubleEncoding) {
     this.defaultDoubleEncoding = TSEncoding.valueOf(defaultDoubleEncoding);
   }
 
@@ -1217,7 +1228,7 @@ public class IoTDBConfig {
     this.defaultTextEncoding = defaultTextEncoding;
   }
 
-  public void setDefaultTextEncoding(String defaultTextEncoding) {
+  void setDefaultTextEncoding(String defaultTextEncoding) {
     this.defaultTextEncoding = TSEncoding.valueOf(defaultTextEncoding);
   }
 
@@ -1229,27 +1240,27 @@ public class IoTDBConfig {
     this.systemFileStorageFs = FSType.valueOf(systemFileStorageFs);
   }
 
-  public FSType getTsFileStorageFs() {
+  FSType getTsFileStorageFs() {
     return tsFileStorageFs;
   }
 
-  public void setTsFileStorageFs(String tsFileStorageFs) {
+  void setTsFileStorageFs(String tsFileStorageFs) {
     this.tsFileStorageFs = FSType.valueOf(tsFileStorageFs);
   }
 
-  public String getCoreSitePath() {
+  String getCoreSitePath() {
     return coreSitePath;
   }
 
-  public void setCoreSitePath(String coreSitePath) {
+  void setCoreSitePath(String coreSitePath) {
     this.coreSitePath = coreSitePath;
   }
 
-  public String getHdfsSitePath() {
+  String getHdfsSitePath() {
     return hdfsSitePath;
   }
 
-  public void setHdfsSitePath(String hdfsSitePath) {
+  void setHdfsSitePath(String hdfsSitePath) {
     this.hdfsSitePath = hdfsSitePath;
   }
 
@@ -1261,15 +1272,15 @@ public class IoTDBConfig {
     return hdfsIp;
   }
 
-  public void setHdfsIp(String[] hdfsIp) {
+  void setHdfsIp(String[] hdfsIp) {
     this.hdfsIp = String.join(",", hdfsIp);
   }
 
-  public String getHdfsPort() {
+  String getHdfsPort() {
     return hdfsPort;
   }
 
-  public void setHdfsPort(String hdfsPort) {
+  void setHdfsPort(String hdfsPort) {
     this.hdfsPort = hdfsPort;
   }
 
@@ -1277,15 +1288,15 @@ public class IoTDBConfig {
     return upgradeThreadNum;
   }
 
-  public void setUpgradeThreadNum(int upgradeThreadNum) {
+  void setUpgradeThreadNum(int upgradeThreadNum) {
     this.upgradeThreadNum = upgradeThreadNum;
   }
 
-  public String getDfsNameServices() {
+  String getDfsNameServices() {
     return dfsNameServices;
   }
 
-  public void setDfsNameServices(String dfsNameServices) {
+  void setDfsNameServices(String dfsNameServices) {
     this.dfsNameServices = dfsNameServices;
   }
 
@@ -1297,47 +1308,47 @@ public class IoTDBConfig {
     return dfsHaNamenodes;
   }
 
-  public void setDfsHaNamenodes(String[] dfsHaNamenodes) {
+  void setDfsHaNamenodes(String[] dfsHaNamenodes) {
     this.dfsHaNamenodes = String.join(",", dfsHaNamenodes);
   }
 
-  public boolean isDfsHaAutomaticFailoverEnabled() {
+  boolean isDfsHaAutomaticFailoverEnabled() {
     return dfsHaAutomaticFailoverEnabled;
   }
 
-  public void setDfsHaAutomaticFailoverEnabled(boolean dfsHaAutomaticFailoverEnabled) {
+  void setDfsHaAutomaticFailoverEnabled(boolean dfsHaAutomaticFailoverEnabled) {
     this.dfsHaAutomaticFailoverEnabled = dfsHaAutomaticFailoverEnabled;
   }
 
-  public String getDfsClientFailoverProxyProvider() {
+  String getDfsClientFailoverProxyProvider() {
     return dfsClientFailoverProxyProvider;
   }
 
-  public void setDfsClientFailoverProxyProvider(String dfsClientFailoverProxyProvider) {
+  void setDfsClientFailoverProxyProvider(String dfsClientFailoverProxyProvider) {
     this.dfsClientFailoverProxyProvider = dfsClientFailoverProxyProvider;
   }
 
-  public boolean isUseKerberos() {
+  boolean isUseKerberos() {
     return useKerberos;
   }
 
-  public void setUseKerberos(boolean useKerberos) {
+  void setUseKerberos(boolean useKerberos) {
     this.useKerberos = useKerberos;
   }
 
-  public String getKerberosKeytabFilePath() {
+  String getKerberosKeytabFilePath() {
     return kerberosKeytabFilePath;
   }
 
-  public void setKerberosKeytabFilePath(String kerberosKeytabFilePath) {
+  void setKerberosKeytabFilePath(String kerberosKeytabFilePath) {
     this.kerberosKeytabFilePath = kerberosKeytabFilePath;
   }
 
-  public String getKerberosPrincipal() {
+  String getKerberosPrincipal() {
     return kerberosPrincipal;
   }
 
-  public void setKerberosPrincipal(String kerberosPrincipal) {
+  void setKerberosPrincipal(String kerberosPrincipal) {
     this.kerberosPrincipal = kerberosPrincipal;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java b/server/src/main/java/org/apache/iotdb/db/conf/ServerConfigConsistent.java
similarity index 76%
copy from server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java
copy to server/src/main/java/org/apache/iotdb/db/conf/ServerConfigConsistent.java
index d4058ba..18ad0a4 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/ServerConfigConsistent.java
@@ -15,17 +15,23 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
 
-package org.apache.iotdb.db.utils;
+package org.apache.iotdb.db.conf;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
-@Target({ElementType.METHOD, ElementType.CONSTRUCTOR})
+/**
+ * Configurations using this annotation should be the same across all nodes in a cluster.
+ */
+@Target({ElementType.FIELD})
 @Retention(RetentionPolicy.SOURCE)
-public @interface TestOnly {
+public @interface ServerConfigConsistent {
+  //TODO#IOTDB-436: the restarted server should check the
+  // configuration consistency as before.
+
 }
+
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 cf99279..ac23eef 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
@@ -23,8 +23,10 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
@@ -38,6 +40,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.conf.ServerConfigConsistent;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy.DirectFlushPolicy;
@@ -91,15 +94,41 @@ public class StorageEngine implements IService {
   private static final ExecutorService recoveryThreadPool = IoTDBThreadPoolFactory
       .newFixedThreadPool(Runtime.getRuntime().availableProcessors(), "Recovery-Thread-Pool");
 
-  private static final StorageEngine INSTANCE = new StorageEngine();
+  static class InstanceHolder {
+    private static final StorageEngine INSTANCE = new StorageEngine();
+  }
 
   public static StorageEngine getInstance() {
-    return INSTANCE;
+    return InstanceHolder.INSTANCE;
   }
 
   private ScheduledExecutorService ttlCheckThread;
   private TsFileFlushPolicy fileFlushPolicy = new DirectFlushPolicy();
 
+  /**
+   * Time range for dividing storage group, the time unit is the same with IoTDB's TimestampPrecision
+   */
+  @ServerConfigConsistent
+  static long timePartitionInterval;
+  static {
+    // build time Interval to divide time partition
+    String timePrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision();
+    switch (timePrecision) {
+      case "ns":
+        timePartitionInterval = IoTDBDescriptor.getInstance().
+            getConfig().getPartitionInterval() * 1000_000_000L;
+        break;
+      case "us":
+        timePartitionInterval = IoTDBDescriptor.getInstance().
+            getConfig().getPartitionInterval() * 1000_000L;
+        break;
+      default:
+        timePartitionInterval = IoTDBDescriptor.getInstance().
+            getConfig().getPartitionInterval() * 1000;
+        break;
+    }
+  }
+
   private StorageEngine() {
     logger = LoggerFactory.getLogger(StorageEngine.class);
     systemDir = FilePathUtils.regularizePath(config.getSystemDir()) + "storage_groups";
@@ -109,7 +138,6 @@ public class StorageEngine implements IService {
     } catch (IOException e) {
       throw new StorageEngineFailureException(e);
     }
-
     // recover upgrade process
     UpgradeUtils.recoverUpgrade();
     /*
@@ -160,14 +188,16 @@ public class StorageEngine implements IService {
   @Override
   public void stop() {
     syncCloseAllProcessor();
-    ttlCheckThread.shutdownNow();
+    if (ttlCheckThread != null) {
+      ttlCheckThread.shutdownNow();
+      try {
+        ttlCheckThread.awaitTermination(30, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        logger.warn("TTL check thread still doesn't exit after 30s");
+      }
+    }
     recoveryThreadPool.shutdownNow();
     this.reset();
-    try {
-      ttlCheckThread.awaitTermination(30, TimeUnit.SECONDS);
-    } catch (InterruptedException e) {
-      logger.warn("TTL check thread still doesn't exit after 30s");
-    }
   }
 
   @Override
@@ -176,7 +206,7 @@ public class StorageEngine implements IService {
   }
 
   public StorageGroupProcessor getProcessor(String path) throws StorageEngineException {
-    String storageGroupName = "";
+    String storageGroupName;
     try {
       storageGroupName = MManager.getInstance().getStorageGroupName(path);
       StorageGroupProcessor processor;
@@ -490,4 +520,37 @@ public class StorageEngine implements IService {
     return file.getParentFile().getParentFile().getName();
   }
 
+  /**
+   *
+   * @return TsFiles (seq or unseq) grouped by their storage group.
+   */
+  public Map<String, List<TsFileResource>> getAllClosedStorageGroupTsFile() {
+    Map<String, List<TsFileResource>> ret = new HashMap<>();
+    for (Entry<String, StorageGroupProcessor> entry : processorMap
+        .entrySet()) {
+      ret.computeIfAbsent(entry.getKey(), sg -> new ArrayList<>()).addAll(entry.getValue().getSequenceFileTreeSet());
+      ret.get(entry.getKey()).addAll(entry.getValue().getUnSequenceFileList());
+      ret.get(entry.getKey()).removeIf(file -> !file.isClosed());
+    }
+    return ret;
+  }
+
+  public void setFileFlushPolicy(TsFileFlushPolicy fileFlushPolicy) {
+    this.fileFlushPolicy = fileFlushPolicy;
+  }
+
+  public boolean isFileAlreadyExist(TsFileResource tsFileResource, String storageGroup) {
+    // TODO-Cluster#350: integrate with time partitioning
+    StorageGroupProcessor processor = processorMap.get(storageGroup);
+    return processor != null && processor.isFileAlreadyExist(tsFileResource);
+  }
+
+  public static long getTimePartitionInterval() {
+    return timePartitionInterval;
+  }
+
+  public static long fromTimeToTimePartition(long time) {
+
+    return time / timePartitionInterval;
+  }
 }
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 edaa0c8..fca11ae 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
@@ -44,6 +44,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
 import org.apache.iotdb.db.engine.merge.manage.MergeManager;
@@ -75,7 +76,6 @@ import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryFileManager;
 import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
-import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.db.utils.UpgradeUtils;
 import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -143,10 +143,7 @@ public class StorageGroupProcessor {
    * time partition id in the storage group -> tsFileProcessor for this time partition
    */
   private final TreeMap<Long, TsFileProcessor> workUnsequenceTsFileProcessors = new TreeMap<>();
-  /**
-   * Time range for dividing storage group, unit is second
-   */
-  private long partitionIntervalForStorageGroup;
+
   /**
    * the schema of time series that belong this storage group
    */
@@ -232,23 +229,6 @@ public class StorageGroupProcessor {
           storageGroupSysDir.getPath());
     }
 
-    // build time Interval to divide time partition
-    String timePrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision();
-    switch (timePrecision) {
-      case "ns":
-        partitionIntervalForStorageGroup = IoTDBDescriptor.getInstance().
-            getConfig().getPartitionInterval() * 1000_000_000L;
-        break;
-      case "us":
-        partitionIntervalForStorageGroup = IoTDBDescriptor.getInstance().
-            getConfig().getPartitionInterval() * 1000_000L;
-        break;
-      default:
-        partitionIntervalForStorageGroup = IoTDBDescriptor.getInstance().
-            getConfig().getPartitionInterval() * 1000;
-        break;
-    }
-
     recover();
   }
 
@@ -317,7 +297,7 @@ public class StorageGroupProcessor {
     // just find any time of device
     Iterator<Long> iterator = startTimeMap.values().iterator();
     if (iterator.hasNext()) {
-      return fromTimeToTimePartition(iterator.next());
+      return StorageEngine.fromTimeToTimePartition(iterator.next());
     }
 
     return -1;
@@ -490,7 +470,7 @@ public class StorageGroupProcessor {
     writeLock();
     try {
       // init map
-      long timePartitionId = fromTimeToTimePartition(insertPlan.getTime());
+      long timePartitionId = StorageEngine.fromTimeToTimePartition(insertPlan.getTime());
       latestTimeForEachDevice.computeIfAbsent(timePartitionId, l -> new HashMap<>())
           .putIfAbsent(insertPlan.getDeviceId(), Long.MIN_VALUE);
       latestFlushedTimeForEachDevice.computeIfAbsent(timePartitionId, id -> new HashMap<>())
@@ -531,7 +511,7 @@ public class StorageGroupProcessor {
       // before is first start point
       int before = loc;
       // before time partition
-      long beforeTimePartition = fromTimeToTimePartition(batchInsertPlan.getTimes()[before]);
+      long beforeTimePartition = StorageEngine.fromTimeToTimePartition(batchInsertPlan.getTimes()[before]);
       // init map
       long lastFlushTime = latestFlushedTimeForEachDevice.
           computeIfAbsent(beforeTimePartition, id -> new HashMap<>()).
@@ -540,7 +520,7 @@ public class StorageGroupProcessor {
       boolean isSequence = false;
       while (loc < batchInsertPlan.getRowCount()) {
         long time = batchInsertPlan.getTimes()[loc];
-        long curTimePartition = fromTimeToTimePartition(time);
+        long curTimePartition = StorageEngine.fromTimeToTimePartition(time);
         results[loc] = TSStatusCode.SUCCESS_STATUS.getStatusCode();
         // start next partition
         if (curTimePartition != beforeTimePartition) {
@@ -636,7 +616,7 @@ public class StorageGroupProcessor {
       throws QueryProcessException {
     TsFileProcessor tsFileProcessor;
     boolean result;
-    long timePartitionId = fromTimeToTimePartition(insertPlan.getTime());
+    long timePartitionId = StorageEngine.fromTimeToTimePartition(insertPlan.getTime());
 
     tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence);
 
@@ -735,10 +715,6 @@ public class StorageGroupProcessor {
   }
 
 
-  private long fromTimeToTimePartition(long time) {
-
-    return time / partitionIntervalForStorageGroup;
-  }
 
   private TsFileProcessor createTsFileProcessor(boolean sequence, long timePartitionId)
       throws IOException, DiskSpaceInsufficientException {
@@ -784,6 +760,7 @@ public class StorageGroupProcessor {
   }
 
   private String getNewTsFileName(long time, long version, int mergeCnt) {
+    allDirectFileVersions.add(version);
     return time + IoTDBConstant.TSFILE_NAME_SEPARATOR + version
         + IoTDBConstant.TSFILE_NAME_SEPARATOR + mergeCnt + TSFILE_SUFFIX;
   }
@@ -1165,7 +1142,7 @@ public class StorageGroupProcessor {
       }
 
       // time partition to divide storage group
-      long timePartitionId = fromTimeToTimePartition(timestamp);
+      long timePartitionId = StorageEngine.fromTimeToTimePartition(timestamp);
       // write log
       if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
         DeletePlan deletionPlan = new DeletePlan(timestamp, new Path(deviceId, measurementId));
@@ -1616,6 +1593,41 @@ public class StorageGroupProcessor {
   }
 
   /**
+   * If the historical versions of a file is a sub-set of the given file's, remove it to reduce
+   * unnecessary merge. Only used when the file sender and the receiver share the same file
+   * close policy.
+   * @param resource
+   */
+  public void removeFullyOverlapFiles(TsFileResource resource) {
+    writeLock();
+    closeQueryLock.writeLock().lock();
+    try {
+      Iterator<TsFileResource> iterator = sequenceFileTreeSet.iterator();
+      removeFullyOverlapFiles(resource, iterator);
+
+      iterator = unSequenceFileList.iterator();
+      removeFullyOverlapFiles(resource, iterator);
+    } finally {
+      closeQueryLock.writeLock().unlock();
+      writeUnlock();
+    }
+  }
+
+  private void removeFullyOverlapFiles(TsFileResource resource, Iterator<TsFileResource> iterator) {
+    while (iterator.hasNext()) {
+      TsFileResource seqFile = iterator.next();
+      if (resource.getHistoricalVersions().containsAll(seqFile.getHistoricalVersions())
+          && !resource.getHistoricalVersions().equals(seqFile.getHistoricalVersions())) {
+        if (seqFile.getWriteQueryLock().writeLock().tryLock()) {
+          iterator.remove();
+          seqFile.remove();
+          seqFile.getWriteQueryLock().writeLock().unlock();
+        }
+      }
+    }
+  }
+
+  /**
    * Get an appropriate filename to ensure the order between files. The tsfile is named after
    * ({systemTime}-{versionNum}-{mergeNum}.tsfile).
    * <p>
@@ -1679,7 +1691,7 @@ public class StorageGroupProcessor {
     for (Entry<String, Long> entry : newTsFileResource.getEndTimeMap().entrySet()) {
       String device = entry.getKey();
       long endTime = newTsFileResource.getEndTimeMap().get(device);
-      long timePartitionId = fromTimeToTimePartition(endTime);
+      long timePartitionId = StorageEngine.fromTimeToTimePartition(endTime);
       if (!latestTimeForEachDevice.computeIfAbsent(timePartitionId, id -> new HashMap<>())
           .containsKey(device)
           || latestTimeForEachDevice.get(timePartitionId).get(device) < endTime) {
@@ -1692,7 +1704,7 @@ public class StorageGroupProcessor {
       if (!latestFlushTimeForPartition.containsKey(device)
           || latestFlushTimeForPartition.get(device) < endTime) {
         latestFlushedTimeForEachDevice
-            .computeIfAbsent(timePartitionId, id -> new HashMap<String, Long>())
+            .computeIfAbsent(timePartitionId, id -> new HashMap<>())
             .put(device, endTime);
       }
     }
@@ -1709,7 +1721,7 @@ public class StorageGroupProcessor {
       TsFileResource tsFileResource)
       throws TsFileProcessorException, DiskSpaceInsufficientException {
     File targetFile;
-    long timeRangeId = fromTimeToTimePartition(
+    long timeRangeId = StorageEngine.fromTimeToTimePartition(
         tsFileResource.getStartTimeMap().entrySet().iterator().next().getValue());
     switch (type) {
       case LOAD_UNSEQUENCE:
@@ -1889,12 +1901,10 @@ public class StorageGroupProcessor {
     checkFilesTTL();
   }
 
-  @TestOnly
   public List<TsFileResource> getSequenceFileTreeSet() {
     return new ArrayList<>(sequenceFileTreeSet);
   }
 
-  @TestOnly
   public List<TsFileResource> getUnSequenceFileList() {
     return unSequenceFileList;
   }
@@ -1909,10 +1919,17 @@ public class StorageGroupProcessor {
     void call(TsFileProcessor caller) throws TsFileProcessorException, IOException;
   }
 
+  public String getStorageGroupName() {
+    return storageGroupName;
+  }
+
+  public boolean isFileAlreadyExist(TsFileResource tsFileResource) {
+    return allDirectFileVersions.containsAll(tsFileResource.getHistoricalVersions());
+  }
+
   @FunctionalInterface
   public interface UpdateEndTimeCallBack {
 
     boolean call(TsFileProcessor caller);
   }
-
 }
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 e9356db..58ecd56 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
@@ -60,12 +60,12 @@ public class TsFileResource {
   /**
    * device -> start time
    */
-  private Map<String, Long> startTimeMap;
+  protected Map<String, Long> startTimeMap;
 
   /**
    * device -> end time. It is null if it's an unsealed sequence tsfile
    */
-  private Map<String, Long> endTimeMap;
+  protected Map<String, Long> endTimeMap;
 
   public TsFileProcessor getProcessor() {
     return processor;
@@ -101,6 +101,25 @@ public class TsFileResource {
 
   private FSFactory fsFactory = FSFactoryProducer.getFSFactory();
 
+  public TsFileResource() {
+  }
+
+  public TsFileResource(TsFileResource other) {
+    this.file = other.file;
+    this.startTimeMap = other.startTimeMap;
+    this.endTimeMap = other.endTimeMap;
+    this.processor = other.processor;
+    this.modFile = other.modFile;
+    this.closed = other.closed;
+    this.deleted = other.deleted;
+    this.isMerging = other.isMerging;
+    this.chunkMetaDataList = other.chunkMetaDataList;
+    this.readOnlyMemChunk = other.readOnlyMemChunk;
+    this.writeQueryLock = other.writeQueryLock;
+    this.fsFactory = other.fsFactory;
+    this.historicalVersions = other.historicalVersions;
+  }
+
   /**
    * for sealed TsFile, call setClosed to close TsFileResource
    */
@@ -162,7 +181,7 @@ public class TsFileResource {
     fsFactory.moveFile(src, dest);
   }
 
-  public void deSerialize() throws IOException {
+  public void deserialize() throws IOException {
     try (InputStream inputStream = fsFactory.getBufferedInputStream(
         file + RESOURCE_SUFFIX)) {
       int size = ReadWriteIOUtils.readInt(inputStream);
@@ -363,11 +382,19 @@ public class TsFileResource {
     return false;
   }
 
+  protected void setStartTimeMap(Map<String, Long> startTimeMap) {
+    this.startTimeMap = startTimeMap;
+  }
+
+  protected void setEndTimeMap(Map<String, Long> endTimeMap) {
+    this.endTimeMap = endTimeMap;
+  }
+
   /**
    * set a file flag indicating that the file is being closed, so during recovery we could know we
    * should close the file.
    */
-  public void setCloseFlag() {
+  void setCloseFlag() {
     try {
       new File(file.getAbsoluteFile() + CLOSING_SUFFIX).createNewFile();
     } catch (IOException e) {
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java b/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java
index 7944bd4..bd03a29 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java
@@ -24,6 +24,10 @@ public class StorageEngineException extends IoTDBException {
 
   private static final long serialVersionUID = 9001649171768311032L;
 
+  public StorageEngineException(Throwable cause) {
+    super(cause, TSStatusCode.STORAGE_ENGINE_ERROR.getStatusCode());
+  }
+
   public StorageEngineException(String message) {
     super(message, TSStatusCode.STORAGE_ENGINE_ERROR.getStatusCode());
   }
@@ -33,6 +37,6 @@ public class StorageEngineException extends IoTDBException {
   }
 
   public StorageEngineException(IoTDBException e) {
-    super(e.getMessage(), e.getErrorCode());
+    super(e, e.getErrorCode());
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java
index 8f9ee27..da40603 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java
@@ -15,7 +15,6 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
 
 package org.apache.iotdb.db.exception.metadata;
@@ -23,11 +22,10 @@ package org.apache.iotdb.db.exception.metadata;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 public class IllegalPathException extends MetadataException {
-
   private static final long serialVersionUID = 2693272249167539978L;
 
   public IllegalPathException(String path) {
     super(String.format("%s is not a legal path", path));
     errorCode = TSStatusCode.PATH_ILLEGAL.getStatusCode();
   }
-}
\ No newline at end of file
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java
index dfca520..5d8921c 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java
@@ -29,6 +29,10 @@ public class MetadataException extends IoTDBException {
 
   private static final long serialVersionUID = 3415275599091623570L;
 
+  public MetadataException(Throwable cause) {
+    super(cause, TSStatusCode.METADATA_ERROR.getStatusCode());
+  }
+
   public MetadataException(Throwable cause, int errorCode) {
     super(cause, TSStatusCode.METADATA_ERROR.getStatusCode());
     this.errorCode = errorCode;
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java
index fe8e476..7d86ade 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java
@@ -15,15 +15,12 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
 
 package org.apache.iotdb.db.exception.metadata;
 
 public class PathNotExistException extends MetadataException {
 
-  private static final long serialVersionUID = 2693272249167539978L;
-
   public PathNotExistException(String path) {
     super(String.format("Path [%s] does not exist", path));
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java
index c43884a..b196309 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java
@@ -15,7 +15,6 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
 
 package org.apache.iotdb.db.exception.metadata;
@@ -27,4 +26,4 @@ public class StorageGroupAlreadySetException extends MetadataException {
   public StorageGroupAlreadySetException(String path) {
     super(String.format("%s has already been set to storage group", path));
   }
-}
\ No newline at end of file
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java b/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
index 9372d15..f8c007c 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java
@@ -35,6 +35,6 @@ public class QueryProcessException extends IoTDBException {
   }
 
   public QueryProcessException(IoTDBException e) {
-    super(e.getMessage(), e.getErrorCode());
+    super(e, e.getErrorCode());
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java b/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java
index 934ece6..aab283c 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java
@@ -24,10 +24,10 @@ public class StorageEngineFailureException extends RuntimeException {
   private static final long serialVersionUID = -1197701024139022020L;
 
   public StorageEngineFailureException(Throwable e) {
-    super("Create system directory failed! " + e.getMessage());
+    super("Create system directory failed! ", e);
   }
 
   public StorageEngineFailureException(String message, Throwable e) {
-    super(message + e.getMessage());
+    super(message, e);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java b/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java
index 249be85..9bae99d 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java
@@ -26,7 +26,7 @@ public class StorageGroupProcessorException extends IoTDBException {
   private static final long serialVersionUID = 7373978140952977661L;
 
   public StorageGroupProcessorException(Exception exception) {
-    super(exception.getMessage(), TSStatusCode.STORAGE_GROUP_PROCESSOR_ERROR.getStatusCode());
+    super(exception, TSStatusCode.STORAGE_GROUP_PROCESSOR_ERROR.getStatusCode());
   }
 
   public StorageGroupProcessorException(String message) {
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 0ef29eb..b34a6a7 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
@@ -24,9 +24,12 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -39,9 +42,11 @@ import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.exception.ConfigAdjusterException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.mnode.LeafMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
 import org.apache.iotdb.db.monitor.MonitorConstants;
@@ -841,4 +846,67 @@ public class MManager {
       lock.readLock().unlock();
     }
   }
+
+  public void collectSeries(MNode startingNode, Collection<MeasurementSchema> timeseriesSchemas) {
+    Deque<MNode> nodeDeque = new ArrayDeque<>();
+    nodeDeque.addLast(startingNode);
+    while (!nodeDeque.isEmpty()) {
+      MNode node = nodeDeque.removeFirst();
+      if (node instanceof LeafMNode) {
+        MeasurementSchema nodeSchema = node.getSchema();
+        timeseriesSchemas.add(new MeasurementSchema(node.getFullPath(),
+            nodeSchema.getType(), nodeSchema.getEncodingType(), nodeSchema.getCompressor()));
+      } else if (!node.getChildren().isEmpty()) {
+        nodeDeque.addAll(node.getChildren().values());
+      }
+    }
+  }
+
+  public void collectSeries(String startingPath, List<MeasurementSchema> timeseriesSchemas) {
+    MNode mNode;
+    try {
+      mNode = getNodeByPath(startingPath);
+    } catch (MetadataException e) {
+      return;
+    }
+    collectSeries(mNode, timeseriesSchemas);
+  }
+
+  /**
+   * For a path, infer all storage groups it may belong to.
+   * The path can have wildcards.
+   * Consider the path into two parts: (1) the sub path which can not contain a storage group name and
+   * (2) the sub path which is substring that begin after the storage group name.
+   * (1) Suppose the part of the path can not contain a storage group name (e.g.,
+   * "root".contains("root.sg") == false), then:
+   * If the wildcard is not at the tail, then for each wildcard, only one level will be inferred
+   * and the wildcard will be removed.
+   * If the wildcard is at the tail, then the inference will go on until the storage groups are found
+   * and the wildcard will be kept.
+   * (2) Suppose the path of the path is a substring that begin after the storage group name. (e.g.,
+   *  For "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*").
+   *  For this part, keep what it is.
+   *
+   * Assuming we have three SGs: root.group1, root.group2, root.area1.group3
+   * Eg1:
+   *  for input "root.*", returns ("root.group1", "root.group1.*"), ("root.group2", "root.group2.*")
+   *  ("root.area1.group3", "root.area1.group3.*")
+   * Eg2:
+   *  for input "root.*.s1", returns ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1")
+   *
+   * Eg3:
+   *  for input "root.area1.*", returns ("root.area1.group3", "root.area1.group3.*")
+   *
+   *
+   * @param path can be a prefix or a full path.
+   * @return StorageGroupName-FullPath pairs
+   */
+  public Map<String, String> determineStorageGroup(String path) throws IllegalPathException {
+    lock.readLock().lock();
+    try {
+      return mtree.determineStorageGroup(path);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 9a07c20..87b6afd 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.regex.Pattern;
@@ -231,6 +232,7 @@ public class MTree implements Serializable {
    *
    * @param path Format: root.node(.node)+
    */
+
   String deleteTimeseriesAndReturnEmptyStorageGroup(String path) throws MetadataException {
     MNode curNode = getNodeByPath(path);
     if (!(curNode instanceof LeafMNode)) {
@@ -755,4 +757,63 @@ public class MTree implements Serializable {
     }
     return res;
   }
+
+  Map<String, String> determineStorageGroup(String path) throws IllegalPathException {
+    Map<String, String> paths = new HashMap<>();
+    String[] nodes = MetaUtils.getNodeNames(path);
+    if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
+      throw new IllegalPathException(path);
+    }
+
+    Deque<MNode> nodeStack = new ArrayDeque<>();
+    Deque<Integer> depthStack = new ArrayDeque<>();
+    if (!root.getChildren().isEmpty()) {
+      nodeStack.push(root);
+      depthStack.push(0);
+    }
+
+    while (!nodeStack.isEmpty()) {
+      MNode mNode = nodeStack.removeFirst();
+      int depth = depthStack.removeFirst();
+
+      determineStorageGroup(depth + 1, nodes, mNode, paths, nodeStack, depthStack);
+    }
+    return paths;
+  }
+
+  /**
+   * Try determining the storage group using the children of a mNode. If one child is a storage
+   * group node, put a storageGroupName-fullPath pair into paths. Otherwise put the children that
+   * match the path into the queue and discard other children.
+   */
+  private void determineStorageGroup(int depth, String[] nodes, MNode mNode,
+      Map<String, String> paths, Deque<MNode> nodeStack, Deque<Integer> depthStack) {
+    String currNode = depth >= nodes.length ? PATH_WILDCARD : nodes[depth];
+    for (Entry<String, MNode> entry : mNode.getChildren().entrySet()) {
+      if (!currNode.equals(PATH_WILDCARD) && !currNode.equals(entry.getKey())) {
+        continue;
+      }
+      // this child is desired
+      MNode child = entry.getValue();
+      if (child instanceof StorageGroupMNode) {
+        // we have found one storage group, record it
+        String sgName = child.getFullPath();
+        // concat the remaining path with the storage group name
+        StringBuilder pathWithKnownSG = new StringBuilder(sgName);
+        for (int i = depth + 1; i < nodes.length; i++) {
+          pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(nodes[i]);
+        }
+        if (depth >= nodes.length - 1 && currNode.equals(PATH_WILDCARD)) {
+          // the we find the sg at the last node and the last node is a wildcard (find "root
+          // .group1", for "root.*"), also append the wildcard (to make "root.group1.*")
+          pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(PATH_WILDCARD);
+        }
+        paths.put(sgName, pathWithKnownSG.toString());
+      } else if (!child.getChildren().isEmpty()) {
+        // push it back so we can traver its children later
+        nodeStack.push(child);
+        depthStack.push(depth);
+      }
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
index 19a4d69..bd1089f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
@@ -41,7 +41,7 @@ import org.apache.iotdb.db.utils.TestOnly;
  */
 public class Planner {
 
-  private ParseDriver parseDriver;
+  protected ParseDriver parseDriver;
 
   public Planner() {
     this.parseDriver = new ParseDriver();
@@ -70,7 +70,7 @@ public class Planner {
    * @return optimized logical operator
    * @throws LogicalOptimizeException exception in logical optimizing
    */
-  private Operator logicalOptimize(Operator operator)
+  protected Operator logicalOptimize(Operator operator)
       throws LogicalOperatorException {
     switch (operator.getType()) {
       case AUTHOR:
@@ -111,7 +111,7 @@ public class Planner {
    */
   private SFWOperator optimizeSFWOperator(SFWOperator root)
       throws LogicalOperatorException {
-    ConcatPathOptimizer concatPathOptimizer = new ConcatPathOptimizer();
+    ConcatPathOptimizer concatPathOptimizer = getConcatPathOptimizer();
     root = (SFWOperator) concatPathOptimizer.transform(root);
     FilterOperator filter = root.getFilterOperator();
     if (filter == null) {
@@ -127,4 +127,7 @@ public class Planner {
     return root;
   }
 
+  protected ConcatPathOptimizer getConcatPathOptimizer() {
+    return new ConcatPathOptimizer();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java
index ec25529..b2466db 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java
@@ -20,11 +20,13 @@ package org.apache.iotdb.db.qp.executor;
 
 import java.io.IOException;
 import java.sql.SQLException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.*;
+import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
+import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -89,5 +91,4 @@ public interface IPlanExecutor {
    * @return result of each row
    */
   Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProcessException;
-
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 01c72a7..25020d3 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -34,6 +34,7 @@ import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TIMESERIES_ENCODING;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE;
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
 import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
 
 import java.io.File;
@@ -62,7 +63,7 @@ import org.apache.iotdb.db.engine.flush.pool.FlushTaskPoolManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.metadata.mnode.InternalMNode;
@@ -127,9 +128,7 @@ import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
 public class PlanExecutor implements IPlanExecutor {
 
   // for data query
-  private IQueryRouter queryRouter;
-  // for insert
-  private StorageEngine storageEngine;
+  protected IQueryRouter queryRouter;
   // for system schema
   private MManager mManager;
   // for administration
@@ -137,7 +136,6 @@ public class PlanExecutor implements IPlanExecutor {
 
   public PlanExecutor() throws QueryProcessException {
     queryRouter = new QueryRouter();
-    storageEngine = StorageEngine.getInstance();
     mManager = MManager.getInstance();
     try {
       authorizer = LocalFileAuthorizer.getInstance();
@@ -222,7 +220,7 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  private QueryDataSet processDataQuery(QueryPlan queryPlan, QueryContext context)
+  protected QueryDataSet processDataQuery(QueryPlan queryPlan, QueryContext context)
       throws StorageEngineException, QueryFilterOptimizationException, QueryProcessException,
       IOException {
     QueryDataSet queryDataSet;
@@ -316,7 +314,7 @@ public class PlanExecutor implements IPlanExecutor {
     return MManager.getInstance().getAllTimeseriesName(path);
   }
 
-  private List<String> getNodesList(String schemaPattern, int level) throws MetadataException {
+  protected List<String> getNodesList(String schemaPattern, int level) throws MetadataException {
     return MManager.getInstance().getNodesList(schemaPattern, level);
   }
 
@@ -337,7 +335,7 @@ public class PlanExecutor implements IPlanExecutor {
       throws MetadataException {
     ListDataSet listDataSet = new ListDataSet(Collections.singletonList(new Path(COLUMN_DEVICES)),
         Collections.singletonList(TSDataType.TEXT));
-    Set<String> devices = MManager.getInstance().getDevices(showDevicesPlan.getPath().toString());
+    Set<String> devices = getDevices(showDevicesPlan.getPath().toString());
     for (String s : devices) {
       RowRecord record = new RowRecord(0);
       Field field = new Field(TSDataType.TEXT);
@@ -348,10 +346,13 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
+  protected Set<String> getDevices(String path) throws MetadataException {
+    return MManager.getInstance().getDevices(path);
+  }
+
   private QueryDataSet processShowChildPaths(ShowChildPathsPlan showChildPathsPlan)
       throws MetadataException {
-    Set<String> childPathsList = MManager.getInstance()
-        .getChildNodePathInNextLevel(showChildPathsPlan.getPath().toString());
+    Set<String> childPathsList = getPathNextChildren(showChildPathsPlan.getPath().toString());
     ListDataSet listDataSet = new ListDataSet(
         Collections.singletonList(new Path(COLUMN_CHILD_PATHS)),
         Collections.singletonList(TSDataType.TEXT));
@@ -365,6 +366,10 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
+  protected Set<String> getPathNextChildren(String path) throws MetadataException {
+     return MManager.getInstance().getChildNodePathInNextLevel(path);
+  }
+
   private QueryDataSet processShowStorageGroup() {
     ListDataSet listDataSet = new ListDataSet(
         Collections.singletonList(new Path(COLUMN_STORAGE_GROUP)),
@@ -390,8 +395,7 @@ public class PlanExecutor implements IPlanExecutor {
         new Path(COLUMN_TIMESERIES_COMPRESSION)),
         Arrays.asList(TSDataType.TEXT, TSDataType.TEXT, TSDataType.TEXT, TSDataType.TEXT,
             TSDataType.TEXT));
-    List<String[]> timeseriesList = MManager.getInstance()
-        .getAllTimeseriesSchema(timeSeriesPlan.getPath().toString());
+    List<String[]> timeseriesList = getTimeseriesSchemas(timeSeriesPlan.getPath().toString());
     for (String[] list : timeseriesList) {
       RowRecord record = new RowRecord(0);
       for (String s : list) {
@@ -404,6 +408,10 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
+  protected List<String[]> getTimeseriesSchemas(String path) throws MetadataException {
+    return MManager.getInstance().getAllTimeseriesSchema(path);
+  }
+
   private QueryDataSet processShowTTLQuery(ShowTTLPlan showTTLPlan) {
     ListDataSet listDataSet = new ListDataSet(
         Arrays.asList(new Path(COLUMN_STORAGE_GROUP), new Path(COLUMN_TTL))
@@ -503,18 +511,12 @@ public class PlanExecutor implements IPlanExecutor {
     try {
       Set<String> existingPaths = new HashSet<>();
       for (Path p : deletePlan.getPaths()) {
-        existingPaths.addAll(mManager.getAllTimeseriesName(p.getFullPath()));
+        existingPaths.addAll(getPaths(p.getFullPath()));
       }
       if (existingPaths.isEmpty()) {
         throw new QueryProcessException(
             "TimeSeries does not exist and its data cannot be deleted");
       }
-      for (String onePath : existingPaths) {
-        if (!mManager.isPathExist(onePath)) {
-          throw new QueryProcessException(String
-              .format("TimeSeries %s does not exist and its data cannot be deleted", onePath));
-        }
-      }
       for (String path : existingPaths) {
         delete(new Path(path), deletePlan.getDeleteTime());
       }
@@ -611,7 +613,7 @@ public class PlanExecutor implements IPlanExecutor {
                 .createTimeseries(fullPath, schema.getType(), schema.getEncodingType(),
                     schema.getCompressor(), Collections.emptyMap());
             if (result) {
-              storageEngine
+              StorageEngine.getInstance()
                   .addTimeSeries(new Path(fullPath), schema.getType(), schema.getEncodingType(),
                       schema.getCompressor(), Collections.emptyMap());
             }
@@ -684,7 +686,7 @@ public class PlanExecutor implements IPlanExecutor {
             String.format("Time series %s does not exist.", path.getFullPath()));
       }
       mManager.getStorageGroupName(path.getFullPath());
-      storageEngine.delete(deviceId, measurementId, timestamp);
+      StorageEngine.getInstance().delete(deviceId, measurementId, timestamp);
     } catch (MetadataException | StorageEngineException e) {
       throw new QueryProcessException(e);
     }
@@ -704,9 +706,7 @@ public class PlanExecutor implements IPlanExecutor {
         String measurement = measurementList[i];
         if (!node.hasChild(measurement)) {
           if (!IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()) {
-            throw new QueryProcessException(
-                String.format("Current deviceId[%s] does not contain measurement:%s",
-                    deviceId, measurement));
+            throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurement);
           }
           TSDataType dataType = TypeInferenceUtils.getPredictedDataType(strValues[i]);
           Path path = new Path(deviceId, measurement);
@@ -716,7 +716,7 @@ public class PlanExecutor implements IPlanExecutor {
                   TSFileDescriptor.getInstance().getConfig().getCompressor(),
                   Collections.emptyMap());
           if (result) {
-            storageEngine.addTimeSeries(path, dataType, getDefaultEncoding(dataType));
+            StorageEngine.getInstance().addTimeSeries(path, dataType, getDefaultEncoding(dataType));
           }
         }
         MNode measurementNode = node.getChild(measurement);
@@ -728,8 +728,8 @@ public class PlanExecutor implements IPlanExecutor {
         dataTypes[i] = measurementNode.getSchema().getType();
       }
       insertPlan.setDataTypes(dataTypes);
-      storageEngine.insert(insertPlan);
-    } catch (PathException | StorageEngineException | MetadataException e) {
+      StorageEngine.getInstance().insert(insertPlan);
+    } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
     }
   }
@@ -782,7 +782,7 @@ public class PlanExecutor implements IPlanExecutor {
                   TSFileDescriptor.getInstance().getConfig().getCompressor(),
                   Collections.emptyMap());
           if (result) {
-            storageEngine
+            StorageEngine.getInstance()
                 .addTimeSeries(path, dataType, getDefaultEncoding(dataType));
           }
         }
@@ -800,8 +800,7 @@ public class PlanExecutor implements IPlanExecutor {
                   measurementNode.getSchema().getType()));
         }
       }
-      return storageEngine.insertBatch(batchInsertPlan);
-
+      return StorageEngine.getInstance().insertBatch(batchInsertPlan);
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
     }
@@ -890,7 +889,7 @@ public class PlanExecutor implements IPlanExecutor {
       boolean result = mManager
           .createTimeseries(path.getFullPath(), dataType, encoding, compressor, props);
       if (result) {
-        storageEngine.addTimeSeries(path, dataType, encoding, compressor, props);
+        StorageEngine.getInstance().addTimeSeries(path, dataType, encoding, compressor, props);
       }
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
@@ -908,7 +907,7 @@ public class PlanExecutor implements IPlanExecutor {
         emptyStorageGroups.addAll(mManager.deleteTimeseries(path.toString()));
       }
       for (String deleteStorageGroup : emptyStorageGroups) {
-        storageEngine.deleteAllDataFilesInOneStorageGroup(deleteStorageGroup);
+        StorageEngine.getInstance().deleteAllDataFilesInOneStorageGroup(deleteStorageGroup);
       }
     } catch (MetadataException e) {
       throw new QueryProcessException(e);
@@ -916,7 +915,7 @@ public class PlanExecutor implements IPlanExecutor {
     return true;
   }
 
-  private boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan)
+  public boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan)
       throws QueryProcessException {
     Path path = setStorageGroupPlan.getPath();
     try {
@@ -932,7 +931,7 @@ public class PlanExecutor implements IPlanExecutor {
     List<String> deletePathList = new ArrayList<>();
     try {
       for (Path storageGroupPath : deleteStorageGroupPlan.getPaths()) {
-        storageEngine.deleteStorageGroup(storageGroupPath.getFullPath());
+        StorageEngine.getInstance().deleteStorageGroup(storageGroupPath.getFullPath());
         deletePathList.add(storageGroupPath.getFullPath());
       }
       mManager.deleteStorageGroups(deletePathList);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java
index 67f5946..bc22acb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java
@@ -44,16 +44,16 @@ import org.apache.iotdb.tsfile.utils.StringContainer;
 public class FilterOperator extends Operator implements Comparable<FilterOperator> {
 
   // it is the symbol of token. e.g. AND is & and OR is |
-  protected String tokenSymbol;
+  String tokenSymbol;
 
-  protected List<FilterOperator> childOperators;
+  private List<FilterOperator> childOperators;
   // leaf filter operator means it doesn't have left and right child filterOperator. Leaf filter
   // should set FunctionOperator.
-  protected boolean isLeaf = false;
+  protected boolean isLeaf;
   // isSingle being true means all recursive children of this filter belong to one seriesPath.
-  protected boolean isSingle = false;
+  boolean isSingle = false;
   // if isSingle = false, singlePath must be null
-  protected Path singlePath = null;
+  Path singlePath = null;
 
   public FilterOperator(int tokenType) {
     super(tokenType);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
index c86a3e0..91abbb4 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.qp.physical;
 
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
@@ -26,6 +27,8 @@ import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
@@ -38,6 +41,17 @@ public abstract class PhysicalPlan {
   private Operator.OperatorType operatorType;
   private static final int NULL_VALUE_LEN = -1;
 
+  //for cluster mode, whether the plan may be splitted into several sub plans
+  protected boolean canbeSplit = true;
+
+  /**
+   * whether the plan can be split into more than one Plans.
+   * Only used in the cluster mode.
+   */
+  public boolean canbeSplit() {
+    return canbeSplit;
+  }
+
   protected PhysicalPlan(boolean isQuery) {
     this.isQuery = isQuery;
   }
@@ -73,6 +87,10 @@ public abstract class PhysicalPlan {
     isQuery = query;
   }
 
+  public void serializeTo(DataOutputStream stream) throws IOException {
+    throw new UnsupportedOperationException("serialize of unimplemented");
+  }
+
   public void serializeTo(ByteBuffer buffer) {
     throw new UnsupportedOperationException("serialize of unimplemented");
   }
@@ -89,6 +107,14 @@ public abstract class PhysicalPlan {
     }
   }
 
+  protected void putString(DataOutputStream stream, String value) throws IOException {
+    if (value == null) {
+      stream.writeInt(NULL_VALUE_LEN);
+    } else {
+      ReadWriteIOUtils.write(value, stream);
+    }
+  }
+
   protected String readString(ByteBuffer buffer) {
     int valueLen = buffer.getInt();
     if (valueLen == NULL_VALUE_LEN) {
@@ -110,6 +136,7 @@ public abstract class PhysicalPlan {
       }
       PhysicalPlanType type = PhysicalPlanType.values()[typeNum];
       PhysicalPlan plan;
+      // TODO-Cluster: support more plans
       switch (type) {
         case INSERT:
           plan = new InsertPlan();
@@ -123,6 +150,14 @@ public abstract class PhysicalPlan {
           plan = new BatchInsertPlan();
           plan.deserializeFrom(buffer);
           break;
+        case SET_STORAGE_GROUP:
+          plan = new SetStorageGroupPlan();
+          plan.deserializeFrom(buffer);
+          break;
+        case CREATE_TIMESERIES:
+          plan = new CreateTimeSeriesPlan();
+          plan.deserializeFrom(buffer);
+          break;
         default:
           throw new IOException("unrecognized log type " + type);
       }
@@ -131,7 +166,7 @@ public abstract class PhysicalPlan {
   }
 
   public enum PhysicalPlanType {
-    INSERT, DELETE, BATCHINSERT
+    INSERT, DELETE, BATCHINSERT, SET_STORAGE_GROUP, CREATE_TIMESERIES
   }
 
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java
index fe1ee39..2510143 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.qp.physical.crud;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
@@ -37,7 +39,7 @@ public class BatchInsertPlan extends PhysicalPlan {
   private String[] measurements;
   private TSDataType[] dataTypes;
 
-  private long[] times;
+  private long[] times; // times should be sorted. It is done in the session API.
   private ByteBuffer timeBuffer;
 
   private Object[] columns;
@@ -60,6 +62,11 @@ public class BatchInsertPlan extends PhysicalPlan {
     this.deviceId = deviceId;
     setMeasurements(measurements);
   }
+  public BatchInsertPlan(String deviceId, String[] measurements) {
+    super(false, OperatorType.BATCHINSERT);
+    this.deviceId = deviceId;
+    setMeasurements(measurements);
+  }
 
   public BatchInsertPlan(String deviceId, String[] measurements, List<Integer> dataTypes) {
     super(false, OperatorType.BATCHINSERT);
@@ -106,6 +113,85 @@ public class BatchInsertPlan extends PhysicalPlan {
   }
 
   @Override
+  public void serializeTo(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.BATCHINSERT.ordinal();
+    stream.writeByte((byte) type);
+
+    putString(stream, deviceId);
+
+    stream.writeInt(measurements.length);
+    for (String m : measurements) {
+      putString(stream, m);
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      stream.writeShort(dataType.serialize());
+    }
+
+    stream.writeInt(index.size());
+
+    if (timeBuffer == null) {
+      for(int loc : index){
+        stream.writeLong(times[loc]);
+      }
+    } else {
+      stream.write(timeBuffer.array());
+      timeBuffer = null;
+    }
+
+    if (valueBuffer == null) {
+      for (int i = 0; i < measurements.length; i++) {
+        TSDataType dataType = dataTypes[i];
+        switch (dataType) {
+          case INT32:
+            int[] intValues = (int[]) columns[i];
+            for(int loc : index){
+              stream.writeInt(intValues[loc]);
+            }
+            break;
+          case INT64:
+            long[] longValues = (long[]) columns[i];
+            for(int loc : index){
+              stream.writeLong(longValues[loc]);
+            }
+            break;
+          case FLOAT:
+            float[] floatValues = (float[]) columns[i];
+            for(int loc : index){
+              stream.writeFloat(floatValues[loc]);
+            }
+            break;
+          case DOUBLE:
+            double[] doubleValues = (double[]) columns[i];
+            for(int loc : index){
+              stream.writeDouble(doubleValues[loc]);
+            }
+            break;
+          case BOOLEAN:
+            boolean[] boolValues = (boolean[]) columns[i];
+            for(int loc : index){
+              stream.write(BytesUtils.boolToByte(boolValues[loc]));
+            }
+            break;
+          case TEXT:
+            Binary[] binaryValues = (Binary[]) columns[i];
+            for(int loc : index){
+              stream.writeInt(binaryValues[loc].getLength());
+              stream.write(binaryValues[loc].getValues());
+            }
+            break;
+          default:
+            throw new UnSupportedDataTypeException(
+                String.format("Data type %s is not supported.", dataType));
+        }
+      }
+    } else {
+      stream.write(valueBuffer.array());
+      valueBuffer = null;
+    }
+  }
+
+  @Override
   public void serializeTo(ByteBuffer buffer) {
     int type = PhysicalPlanType.BATCHINSERT.ordinal();
     buffer.put((byte) type);
@@ -250,6 +336,10 @@ public class BatchInsertPlan extends PhysicalPlan {
     }
   }
 
+  public void setDataTypes(TSDataType[] dataTypes) {
+    this.dataTypes = dataTypes;
+  }
+
   public Object[] getColumns() {
     return columns;
   }
@@ -303,4 +393,5 @@ public class BatchInsertPlan extends PhysicalPlan {
   public void setRowCount(int size) {
     this.rowCount = size;
   }
+
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
index a46aaea..d2f1236 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.qp.physical.crud;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
@@ -102,6 +104,14 @@ public class DeletePlan extends PhysicalPlan {
   }
 
   @Override
+  public void serializeTo(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.DELETE.ordinal();
+    stream.writeByte((byte) type);
+    stream.writeLong(deleteTime);
+    putString(stream, paths.get(0).getFullPath());
+  }
+
+  @Override
   public void serializeTo(ByteBuffer buffer) {
     int type = PhysicalPlanType.DELETE.ordinal();
     buffer.put((byte) type);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
index 048370b..d2e272b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.qp.physical.crud;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -41,6 +43,7 @@ public class InsertPlan extends PhysicalPlan {
 
   public InsertPlan() {
     super(false, OperatorType.INSERT);
+    canbeSplit = false;
   }
 
   @TestOnly
@@ -50,6 +53,7 @@ public class InsertPlan extends PhysicalPlan {
     this.deviceId = deviceId;
     this.measurements = new String[] {measurement};
     this.values = new String[] {insertValue};
+    canbeSplit = false;
   }
 
   public InsertPlan(TSRecord tsRecord) {
@@ -64,6 +68,7 @@ public class InsertPlan extends PhysicalPlan {
       dataTypes[i] = tsRecord.dataPointList.get(i).getType();
       values[i] = tsRecord.dataPointList.get(i).getValue().toString();
     }
+    canbeSplit = false;
   }
 
   public InsertPlan(String deviceId, long insertTime, String[] measurementList,
@@ -73,6 +78,7 @@ public class InsertPlan extends PhysicalPlan {
     this.deviceId = deviceId;
     this.measurements = measurementList;
     this.values = insertValues;
+    canbeSplit = false;
   }
 
   public long getTime() {
@@ -145,6 +151,25 @@ public class InsertPlan extends PhysicalPlan {
   }
 
   @Override
+  public void serializeTo(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.INSERT.ordinal();
+    stream.writeByte((byte) type);
+    stream.writeLong(time);
+
+    putString(stream, deviceId);
+
+    stream.writeInt(measurements.length);
+    for (String m : measurements) {
+      putString(stream, m);
+    }
+
+    stream.writeInt(values.length);
+    for (String m : values) {
+      putString(stream, m);
+    }
+  }
+
+  @Override
   public void serializeTo(ByteBuffer buffer) {
     int type = PhysicalPlanType.INSERT.ordinal();
     buffer.put((byte) type);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java
index 044d838..b1851ad 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java
@@ -22,6 +22,9 @@ import org.apache.iotdb.tsfile.read.common.Path;
 
 /**
  * CountPlan is used to count time-series and count nodes.
+ * COUNT_TIMESERIES if  using "COUNT TIMESERIES <Path>" and only this command supports wildcard.
+ * COUNT_NODE_TIMESERIES if using "COUNT TIMESERIES <Path> GROUP BY LEVEL=<INTEGER>"
+ * COUNT_NODE if using "COUNT NODES <Path> LEVEL=<INTEGER>"
  */
 public class CountPlan extends ShowPlan {
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
index a6ce3cd..90cb602 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
@@ -18,10 +18,13 @@
  */
 package org.apache.iotdb.db.qp.physical.sys;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-
+import java.util.Objects;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -36,8 +39,13 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
   private TSEncoding encoding;
   private CompressionType compressor;
   private Map<String, String> props;
-	  
-  public CreateTimeSeriesPlan(Path path, TSDataType dataType, TSEncoding encoding, 
+
+  public CreateTimeSeriesPlan() {
+    super(false, Operator.OperatorType.CREATE_TIMESERIES);
+    canbeSplit = false;
+  }
+
+  public CreateTimeSeriesPlan(Path path, TSDataType dataType, TSEncoding encoding,
       CompressionType compressor, Map<String, String> props) {
     super(false, Operator.OperatorType.CREATE_TIMESERIES);
     this.path = path;
@@ -45,6 +53,7 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     this.encoding = encoding;
     this.compressor = compressor;
     this.props = props;
+    canbeSplit = false;
   }
   
   public Path getPath() {
@@ -89,14 +98,8 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
   
   @Override
   public String toString() {
-    String ret = String.format("seriesPath: %s%nresultDataType: %s%nencoding: %s%nnamespace type:"
-        + " ADD_PATH%nargs: ", path, dataType, encoding);
-    StringBuilder stringBuilder = new StringBuilder(ret.length()+50);
-    stringBuilder.append(ret);
-    for (Map.Entry<String, String> prop : props.entrySet()) {
-      stringBuilder.append(prop.getKey()).append("=").append(prop.getValue()).append(",");
-    }
-    return stringBuilder.toString();
+    return String.format("seriesPath: %s, resultDataType: %s, encoding: %s, compression: %s", path,
+        dataType, encoding, compressor);
   }
   
   @Override
@@ -104,4 +107,45 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     return Collections.singletonList(path);
   }
 
+  @Override
+  public void serializeTo(DataOutputStream stream) throws IOException {
+    stream.write(PhysicalPlanType.CREATE_TIMESERIES.ordinal());
+    byte[] pathBytes = path.getFullPath().getBytes();
+    stream.writeInt(pathBytes.length);
+    stream.write(pathBytes);
+    stream.write(dataType.ordinal());
+    stream.write(encoding.ordinal());
+    stream.write(compressor.ordinal());
+  }
+
+  @Override
+  public void deserializeFrom(ByteBuffer buffer) {
+    int length = buffer.getInt();
+    byte[] pathBytes = new byte[length];
+    buffer.get(pathBytes);
+    path = new Path(new String(pathBytes));
+    dataType = TSDataType.values()[buffer.get()];
+    encoding = TSEncoding.values()[buffer.get()];
+    compressor = CompressionType.values()[buffer.get()];
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    CreateTimeSeriesPlan that = (CreateTimeSeriesPlan) o;
+    return Objects.equals(path, that.path) &&
+        dataType == that.dataType &&
+        encoding == that.encoding &&
+        compressor == that.compressor;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(path, dataType, encoding, compressor);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
index 72e9ebc..28d19cd 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java
@@ -18,16 +18,23 @@
  */
 package org.apache.iotdb.db.qp.physical.sys;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
-
+import java.util.Objects;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.tsfile.read.common.Path;
 
 public class SetStorageGroupPlan extends PhysicalPlan {
   private Path path;
-  
+
+  public SetStorageGroupPlan() {
+    super(false, Operator.OperatorType.SET_STORAGE_GROUP);
+  }
+
   public SetStorageGroupPlan(Path path) {
     super(false, Operator.OperatorType.SET_STORAGE_GROUP);
     this.path = path;
@@ -50,4 +57,41 @@ public class SetStorageGroupPlan extends PhysicalPlan {
     return ret;
   }
 
+  @Override
+  public void serializeTo(DataOutputStream stream) throws IOException {
+    stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal());
+    byte[] fullPathBytes = path.getFullPath().getBytes();
+    stream.writeInt(fullPathBytes.length);
+    stream.write(fullPathBytes);
+  }
+
+  @Override
+  public void deserializeFrom(ByteBuffer buffer) {
+    int length = buffer.getInt();
+    byte[] fullPathBytes = new byte[length];
+    buffer.get(fullPathBytes);
+    path = new Path(new String(fullPathBytes));
+  }
+
+  @Override
+  public String toString() {
+    return "SetStorageGroup{" + path + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SetStorageGroupPlan that = (SetStorageGroupPlan) o;
+    return Objects.equals(path, that.path);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(path);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java
index 9f6c199..a1171ae 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java
@@ -22,11 +22,13 @@ package org.apache.iotdb.db.qp.physical.sys;
 import org.apache.iotdb.tsfile.read.common.Path;
 
 public class ShowChildPathsPlan extends ShowPlan {
+  //show child paths SQL does not support path with wildcard
   private Path path;
 
   public ShowChildPathsPlan(ShowContentType showContentType, Path path) {
     super(showContentType);
     this.path = path;
+    canbeSplit = false;
   }
 
   public Path getPath() {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 49a21c2..64805fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -82,7 +82,7 @@ import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
 import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
-import org.apache.iotdb.db.service.TSServiceImpl;
+import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
@@ -210,6 +210,9 @@ public class PhysicalGenerator {
     }
   }
 
+  protected TSDataType getSeriesType(String path) throws MetadataException {
+    return SchemaUtils.getSeriesType(path);
+  }
 
   private PhysicalPlan transformQuery(QueryOperator queryOperator)
       throws QueryProcessException {
@@ -282,8 +285,8 @@ public class PhysicalGenerator {
           Path fullPath = Path.addPrefixPath(suffixPath, device);
           try {
             // remove stars in SELECT to get actual paths
-            List<String> actualPaths = MManager.getInstance()
-                .getAllTimeseriesName(fullPath.getFullPath());
+            List<String> actualPaths = getMatchedTimeseries(fullPath.getFullPath());  // remove stars in SELECT to
+            // get actual paths
 
             // for actual non exist path
             if (actualPaths.isEmpty() && originAggregations.isEmpty()) {
@@ -305,7 +308,7 @@ public class PhysicalGenerator {
                 pathForDataType = path.getFullPath();
                 measurementChecked = path.getMeasurement();
               }
-              TSDataType dataType = TSServiceImpl.getSeriesType(pathForDataType);
+              TSDataType dataType = getSeriesType(pathForDataType);
               if (dataTypeConsistencyChecker.containsKey(measurementChecked)) {
                 if (!dataType.equals(dataTypeConsistencyChecker.get(measurementChecked))) {
                   throw new QueryProcessException(
@@ -421,7 +424,7 @@ public class PhysicalGenerator {
     Set<String> deviceSet = new LinkedHashSet<>();
     try {
       for (Path path : paths) {
-        Set<String> tempDS = MManager.getInstance().getDevices(path.getFullPath());
+        Set<String> tempDS = getMatchedDevices(path.getFullPath());
         deviceSet.addAll(tempDS);
       }
       retDevices = new ArrayList<>(deviceSet);
@@ -454,7 +457,7 @@ public class PhysicalGenerator {
     List<Path> paths = queryPlan.getPaths();
     List<TSDataType> dataTypes = new ArrayList<>(paths.size());
     for (Path path : paths) {
-      TSDataType seriesType = MManager.getInstance().getSeriesType(path.toString());
+      TSDataType seriesType = getSeriesType(path.toString());
       dataTypes.add(seriesType);
       queryPlan.addTypeMapping(path, seriesType);
     }
@@ -522,5 +525,13 @@ public class PhysicalGenerator {
       }
     }
   }
+
+  protected List<String> getMatchedTimeseries(String path) throws MetadataException {
+    return MManager.getInstance().getAllTimeseriesName(path);
+  }
+
+  protected Set<String> getMatchedDevices(String path) throws MetadataException {
+    return MManager.getInstance().getDevices(path);
+  }
 }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
index bd36fa4..91c2e0b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
@@ -18,19 +18,26 @@
  */
 package org.apache.iotdb.db.qp.strategy.optimizer;
 
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.runtime.SQLParserException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.logical.crud.*;
+import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
+import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
+import org.apache.iotdb.db.qp.logical.crud.FromOperator;
+import org.apache.iotdb.db.qp.logical.crud.FunctionOperator;
+import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
+import org.apache.iotdb.db.qp.logical.crud.SFWOperator;
+import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.*;
-
 /**
  * concat paths in select and from clause.
  */
@@ -174,7 +181,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
    * @param seriesLimit is ensured to be positive integer
    * @param seriesOffset is ensured to be non-negative integer
    */
-  public void slimitTrim(SelectOperator select, int seriesLimit, int seriesOffset)
+  private void slimitTrim(SelectOperator select, int seriesLimit, int seriesOffset)
       throws LogicalOptimizeException {
     List<Path> suffixList = select.getSuffixPaths();
     List<String> aggregations = select.getAggregations();
@@ -268,7 +275,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
     LinkedHashMap<String, Integer> pathMap = new LinkedHashMap<>();
     try {
       for (Path path : paths) {
-        List<String> all = MManager.getInstance().getAllTimeseriesName(path.getFullPath());
+        List<String> all = removeWildcard(path.getFullPath());
         for (String subPath : all) {
           if (!pathMap.containsKey(subPath)) {
             pathMap.put(subPath, 1);
@@ -290,7 +297,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
     List<String> newAggregations = new ArrayList<>();
     for (int i = 0; i < paths.size(); i++) {
       try {
-        List<String> actualPaths = MManager.getInstance().getAllTimeseriesName(paths.get(i).getFullPath());
+        List<String> actualPaths = removeWildcard(paths.get(i).getFullPath());
         for (String actualPath : actualPaths) {
           retPaths.add(new Path(actualPath));
           if (afterConcatAggregations != null && !afterConcatAggregations.isEmpty()) {
@@ -304,4 +311,8 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
     selectOperator.setSuffixPathList(retPaths);
     selectOperator.setAggregations(newAggregations);
   }
+
+  protected List<String> removeWildcard(String path) throws MetadataException {
+    return MManager.getInstance().getAllTimeseriesName(path);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java b/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
index 4b88bca..70019f1 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.query.aggregation;
 
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.query.factory.AggregateResultFactory;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
@@ -284,4 +285,8 @@ public abstract class AggregateResult {
     return hasResult;
   }
 
+  @Override
+  public String toString() {
+    return String.valueOf(getResult());
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index 03cbecc..a6d4653 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -105,7 +105,7 @@ public class QueryResourceManager {
         try {
           deserializer.close();
         } catch (IOException e) {
-          throw new StorageEngineException(e.getMessage());
+          throw new StorageEngineException(e);
         }
       }
       externalSortFileMap.remove(queryId);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java
index 20abcd9..40a1226 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java
@@ -201,7 +201,7 @@ public class AlignByDeviceDataSet extends QueryDataSet {
           default:
             throw new IOException("unsupported DataSetType");
         }
-      } catch (QueryProcessException | QueryFilterOptimizationException | StorageEngineException | IOException e) {
+      } catch (QueryProcessException | QueryFilterOptimizationException | StorageEngineException e) {
         throw new IOException(e);
       }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
index 9e94262..208e3f2 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
@@ -21,16 +21,16 @@ package org.apache.iotdb.db.query.dataset;
 import java.io.IOException;
 import java.util.List;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
-import org.apache.iotdb.db.query.timegenerator.ServerTimeGenerator;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Field;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
 
 public class RawQueryDataSetWithValueFilter extends QueryDataSet {
 
-  private ServerTimeGenerator timeGenerator;
+  private TimeGenerator timeGenerator;
   private List<IReaderByTimestamp> seriesReaderByTimestampList;
   private boolean hasCachedRowRecord;
   private RowRecord cachedRowRecord;
@@ -44,7 +44,7 @@ public class RawQueryDataSetWithValueFilter extends QueryDataSet {
    * @param readers       readers in List(IReaderByTimeStamp) structure
    */
   public RawQueryDataSetWithValueFilter(List<Path> paths, List<TSDataType> dataTypes,
-      ServerTimeGenerator timeGenerator, List<IReaderByTimestamp> readers) {
+      TimeGenerator timeGenerator, List<IReaderByTimestamp> readers) {
     super(paths, dataTypes);
     this.timeGenerator = timeGenerator;
     this.seriesReaderByTimestampList = readers;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
index 398ef9e..8ceff22 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.query.dataset.groupby;
 
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
index bf9c5e6..0062e22 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.query.dataset.groupby;
 
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
index c39a392..d21b2d5 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
@@ -36,11 +36,13 @@ import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.dataset.SingleDataSet;
 import org.apache.iotdb.db.query.factory.AggregateResultFactory;
+import org.apache.iotdb.db.query.filter.TsFileFilter;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.series.IAggregateReader;
 import org.apache.iotdb.db.query.reader.series.SeriesAggregateReader;
 import org.apache.iotdb.db.query.reader.series.SeriesReaderByTimestamp;
 import org.apache.iotdb.db.query.timegenerator.ServerTimeGenerator;
+import org.apache.iotdb.db.utils.QueryUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -50,13 +52,14 @@ import org.apache.iotdb.tsfile.read.expression.IExpression;
 import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
 
 public class AggregationExecutor {
 
   private List<Path> selectedSeries;
-  private List<TSDataType> dataTypes;
-  private List<String> aggregations;
-  private IExpression expression;
+  protected List<TSDataType> dataTypes;
+  protected List<String> aggregations;
+  protected IExpression expression;
 
   /**
    * aggregation batch calculation size.
@@ -107,32 +110,45 @@ public class AggregationExecutor {
    * @param context query context
    * @return AggregateResult list
    */
-  private List<AggregateResult> aggregateOneSeries(
+  protected List<AggregateResult> aggregateOneSeries(
       Map.Entry<Path, List<Integer>> pathToAggrIndexes,
       Filter timeFilter, QueryContext context)
       throws IOException, QueryProcessException, StorageEngineException {
     List<AggregateResult> aggregateResultList = new ArrayList<>();
-    List<Boolean> isCalculatedList = new ArrayList<>();
+
     Path seriesPath = pathToAggrIndexes.getKey();
     TSDataType tsDataType = dataTypes.get(pathToAggrIndexes.getValue().get(0));
 
+    for (int i : pathToAggrIndexes.getValue()) {
+      // construct AggregateResult
+      AggregateResult aggregateResult = AggregateResultFactory
+          .getAggrResultByName(aggregations.get(i), tsDataType);
+      aggregateResultList.add(aggregateResult);
+    }
+    aggregateOneSeries(seriesPath, context, timeFilter, tsDataType, aggregateResultList, null);
+    return aggregateResultList;
+  }
+
+  public static void aggregateOneSeries(Path seriesPath, QueryContext context, Filter timeFilter,
+      TSDataType tsDataType, List<AggregateResult> aggregateResultList, TsFileFilter fileFilter)
+      throws StorageEngineException, IOException, QueryProcessException {
+    List<Boolean> isCalculatedList = new ArrayList<>();
+    for (int i = 0; i < aggregateResultList.size(); i++) {
+      isCalculatedList.add(false);
+    }
     // construct series reader without value filter
     QueryDataSource queryDataSource = QueryResourceManager.getInstance()
         .getQueryDataSource(seriesPath, context, timeFilter);
+    if (fileFilter != null) {
+      QueryUtils.filterQueryDataSource(queryDataSource, fileFilter);
+    }
     // update filter by TTL
     timeFilter = queryDataSource.updateFilterUsingTTL(timeFilter);
 
-    IAggregateReader seriesReader = new SeriesAggregateReader(pathToAggrIndexes.getKey(),
+    IAggregateReader seriesReader = new SeriesAggregateReader(seriesPath,
         tsDataType, context, queryDataSource, timeFilter, null, null);
 
-    for (int i : pathToAggrIndexes.getValue()) {
-      // construct AggregateResult
-      AggregateResult aggregateResult = AggregateResultFactory
-          .getAggrResultByName(aggregations.get(i), tsDataType);
-      aggregateResultList.add(aggregateResult);
-      isCalculatedList.add(false);
-    }
-    int remainingToCalculate = pathToAggrIndexes.getValue().size();
+    int remainingToCalculate = aggregateResultList.size();
 
     while (seriesReader.hasNextChunk()) {
       // cal by chunk statistics
@@ -146,7 +162,7 @@ public class AggregationExecutor {
               isCalculatedList.set(i, true);
               remainingToCalculate--;
               if (remainingToCalculate == 0) {
-                return aggregateResultList;
+                return;
               }
             }
           }
@@ -166,7 +182,7 @@ public class AggregationExecutor {
                 isCalculatedList.set(i, true);
                 remainingToCalculate--;
                 if (remainingToCalculate == 0) {
-                  return aggregateResultList;
+                  return;
                 }
               }
             }
@@ -186,7 +202,7 @@ public class AggregationExecutor {
                 isCalculatedList.set(i, true);
                 remainingToCalculate--;
                 if (remainingToCalculate == 0) {
-                  return aggregateResultList;
+                  return;
                 }
               }
             }
@@ -194,7 +210,6 @@ public class AggregationExecutor {
         }
       }
     }
-    return aggregateResultList;
   }
 
   /**
@@ -203,15 +218,14 @@ public class AggregationExecutor {
    * @param context query context.
    */
   public QueryDataSet executeWithValueFilter(QueryContext context)
-      throws StorageEngineException, PathException, IOException {
+      throws StorageEngineException, IOException {
 
-    ServerTimeGenerator timestampGenerator = new ServerTimeGenerator(expression, context);
+    TimeGenerator timestampGenerator = getTimeGenerator(context);
     List<IReaderByTimestamp> readersOfSelectedSeries = new ArrayList<>();
     for (int i = 0; i < selectedSeries.size(); i++) {
       Path path = selectedSeries.get(i);
-      SeriesReaderByTimestamp seriesReaderByTimestamp = new SeriesReaderByTimestamp(path,
-          dataTypes.get(i), context,
-          QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null);
+      IReaderByTimestamp seriesReaderByTimestamp = getReaderByTime(path,
+          dataTypes.get(i), context);
       readersOfSelectedSeries.add(seriesReaderByTimestamp);
     }
 
@@ -225,11 +239,22 @@ public class AggregationExecutor {
     return constructDataSet(aggregateResults);
   }
 
+  protected TimeGenerator getTimeGenerator(QueryContext context) throws StorageEngineException {
+    return new ServerTimeGenerator(expression, context);
+  }
+
+  protected IReaderByTimestamp getReaderByTime(Path path, TSDataType dataType,
+      QueryContext context) throws StorageEngineException {
+    return new SeriesReaderByTimestamp(path,
+        dataType, context,
+        QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null);
+  }
+
   /**
    * calculate aggregation result with value filter.
    */
   private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
-      ServerTimeGenerator timestampGenerator, List<IReaderByTimestamp> readersOfSelectedSeries)
+      TimeGenerator timestampGenerator, List<IReaderByTimestamp> readersOfSelectedSeries)
       throws IOException {
 
     while (timestampGenerator.hasNext()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
index 3f27bb4..f8e1297 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
 import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByWithValueFilterDataSet;
@@ -64,7 +65,7 @@ public class QueryRouter implements IQueryRouter {
     }
     queryPlan.setExpression(optimizedExpression);
 
-    RawDataQueryExecutor rawDataQueryExecutor = new RawDataQueryExecutor(queryPlan);
+    RawDataQueryExecutor rawDataQueryExecutor = getRawDataQueryExecutor(queryPlan);
 
     if (!queryPlan.isAlignByTime()) {
       return rawDataQueryExecutor.executeNonAlign(context);
@@ -78,6 +79,10 @@ public class QueryRouter implements IQueryRouter {
     return rawDataQueryExecutor.executeWithoutValueFilter(context);
   }
 
+  protected RawDataQueryExecutor getRawDataQueryExecutor(RawDataQueryPlan queryPlan) {
+    return new RawDataQueryExecutor(queryPlan);
+  }
+
   @Override
   public QueryDataSet aggregate(AggregationPlan aggregationPlan, QueryContext context)
       throws QueryFilterOptimizationException, StorageEngineException, QueryProcessException,
@@ -93,7 +98,7 @@ public class QueryRouter implements IQueryRouter {
 
     aggregationPlan.setExpression(optimizedExpression);
 
-    AggregationExecutor engineExecutor = new AggregationExecutor(aggregationPlan);
+    AggregationExecutor engineExecutor = getAggregationExecutor(aggregationPlan);
 
     if (optimizedExpression != null
         && optimizedExpression.getType() != ExpressionType.GLOBAL_TIME) {
@@ -103,6 +108,9 @@ public class QueryRouter implements IQueryRouter {
     return engineExecutor.executeWithoutValueFilter(context);
   }
 
+  protected AggregationExecutor getAggregationExecutor(AggregationPlan aggregationPlan) {
+    return new AggregationExecutor(aggregationPlan);
+  }
 
   @Override
   public QueryDataSet groupBy(GroupByPlan groupByPlan, QueryContext context)
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java
index 8785f54..4f9ea11 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java
@@ -25,8 +25,8 @@ import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
-import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithValueFilter;
 import org.apache.iotdb.db.query.dataset.NonAlignEngineDataSet;
+import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithValueFilter;
 import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithoutValueFilter;
 import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.series.ManagedSeriesReader;
@@ -39,15 +39,16 @@ import org.apache.iotdb.tsfile.read.expression.IExpression;
 import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
 
 /**
  * IoTDB query executor.
  */
 public class RawDataQueryExecutor {
 
-  private List<Path> deduplicatedPaths;
-  private List<TSDataType> deduplicatedDataTypes;
-  private IExpression optimizedExpression;
+  protected List<Path> deduplicatedPaths;
+  protected List<TSDataType> deduplicatedDataTypes;
+  protected IExpression optimizedExpression;
 
   public RawDataQueryExecutor(RawDataQueryPlan queryPlan) {
     this.deduplicatedPaths = queryPlan.getDeduplicatedPaths();
@@ -77,7 +78,7 @@ public class RawDataQueryExecutor {
         readersOfSelectedSeries);
   }
 
-  private List<ManagedSeriesReader> initManagedSeriesReader(QueryContext context)
+  protected List<ManagedSeriesReader> initManagedSeriesReader(QueryContext context)
       throws StorageEngineException {
     Filter timeFilter = null;
     if (optimizedExpression != null) {
@@ -108,19 +109,30 @@ public class RawDataQueryExecutor {
    */
   public QueryDataSet executeWithValueFilter(QueryContext context) throws StorageEngineException {
 
-    ServerTimeGenerator timestampGenerator = new ServerTimeGenerator(
+    TimeGenerator timestampGenerator = getTimeGenerator(
         optimizedExpression, context);
 
     List<IReaderByTimestamp> readersOfSelectedSeries = new ArrayList<>();
     for (int i = 0; i < deduplicatedPaths.size(); i++) {
       Path path = deduplicatedPaths.get(i);
-      SeriesReaderByTimestamp seriesReaderByTimestamp = new SeriesReaderByTimestamp(path,
-          deduplicatedDataTypes.get(i), context,
-          QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null);
+      IReaderByTimestamp seriesReaderByTimestamp = getReaderByTimestamp(path,
+          deduplicatedDataTypes.get(i), context);
       readersOfSelectedSeries.add(seriesReaderByTimestamp);
     }
     return new RawQueryDataSetWithValueFilter(deduplicatedPaths, deduplicatedDataTypes,
         timestampGenerator, readersOfSelectedSeries);
   }
 
+  protected IReaderByTimestamp getReaderByTimestamp(Path path, TSDataType dataType,
+      QueryContext context)
+      throws StorageEngineException {
+    return new SeriesReaderByTimestamp(path,
+        dataType, context,
+        QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null);
+  }
+
+  protected TimeGenerator getTimeGenerator(IExpression expression,
+      QueryContext context) throws StorageEngineException {
+    return new ServerTimeGenerator(expression, context);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java
index 4d91289..bf05fba 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java
@@ -26,7 +26,7 @@ import org.apache.iotdb.tsfile.read.reader.IPointReader;
 /**
  * This class represents an external sort job. Every job will use a separated directory.
  */
-public class ExternalSortJob {
+class ExternalSortJob {
 
   private List<ExternalSortJobPart> partList;
 
@@ -41,5 +41,4 @@ public class ExternalSortJob {
     }
     return readers;
   }
-
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java
index 1db8cd3..c1e7071 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java
@@ -16,29 +16,28 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.query.externalsort;
+ package org.apache.iotdb.db.query.externalsort;
 
 
-import java.util.concurrent.atomic.AtomicLong;
+ public class ExternalSortJobScheduler {
 
-public class ExternalSortJobScheduler {
-
-  private AtomicLong jobId = new AtomicLong(0);
+  private long queryId = 0;
 
   private ExternalSortJobScheduler() {
 
   }
 
-  long genJobId() {
-    return jobId.getAndIncrement();
+  synchronized long genJobId() {
+   queryId++;
+   return queryId;
   }
 
   private static class ExternalSortJobSchedulerHelper {
 
-    private static final ExternalSortJobScheduler INSTANCE = new ExternalSortJobScheduler();
+   private static final ExternalSortJobScheduler INSTANCE = new ExternalSortJobScheduler();
   }
 
   public static ExternalSortJobScheduler getInstance() {
-    return ExternalSortJobSchedulerHelper.INSTANCE;
+   return ExternalSortJobSchedulerHelper.INSTANCE;
   }
-}
+ }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java
index 5fba08b..00097a8 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java
@@ -103,10 +103,10 @@ public class SimpleExternalSortEngine implements ExternalSortJobEngine {
         int toIndex = Math.min(i + minExternalSortSourceCount, ret.size());
         List<ExternalSortJobPart> partGroup = ret.subList(i, toIndex);
         i = toIndex;
-        StringBuilder tmpFilePath = new StringBuilder(queryDir).append(jobId).append("_")
-            .append(partId);
+        String tmpFilePath = queryDir + jobId + "_"
+            + partId;
         MultiSourceExternalSortJobPart part = new MultiSourceExternalSortJobPart(queryId,
-            tmpFilePath.toString(), partGroup);
+            tmpFilePath, partGroup);
         tmpPartList.add(part);
         partId++;
       }
@@ -115,19 +115,19 @@ public class SimpleExternalSortEngine implements ExternalSortJobEngine {
     return new ExternalSortJob(ret);
   }
 
-  public String getQueryDir() {
+  String getQueryDir() {
     return queryDir;
   }
 
-  public void setQueryDir(String queryDir) {
+  void setQueryDir(String queryDir) {
     this.queryDir = queryDir;
   }
 
-  public int getMinExternalSortSourceCount() {
+  int getMinExternalSortSourceCount() {
     return minExternalSortSourceCount;
   }
 
-  public void setMinExternalSortSourceCount(int minExternalSortSourceCount) {
+  void setMinExternalSortSourceCount(int minExternalSortSourceCount) {
     this.minExternalSortSourceCount = minExternalSortSourceCount;
   }
 
@@ -171,10 +171,10 @@ public class SimpleExternalSortEngine implements ExternalSortJobEngine {
 
   private static class SimpleExternalSortJobEngineHelper {
 
-    private static SimpleExternalSortEngine INSTANCE = new SimpleExternalSortEngine();
+    private static final SimpleExternalSortEngine INSTANCE = new SimpleExternalSortEngine();
   }
 
   public static SimpleExternalSortEngine getInstance() {
     return SimpleExternalSortJobEngineHelper.INSTANCE;
   }
-}
\ No newline at end of file
+}
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 dd1273f..071974b 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
@@ -42,6 +42,10 @@ public class SeriesReaderByTimestamp implements IReaderByTimestamp {
         dataSource, TimeFilter.gtEq(Long.MIN_VALUE), null, fileFilter);
   }
 
+  public SeriesReaderByTimestamp(SeriesReader seriesReader) {
+    this.seriesReader = seriesReader;
+  }
+
   @Override
   public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
     Object[] result = new Object[timestamps.length];
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
index 6901250..ee5baff 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
@@ -130,7 +130,7 @@ public class PriorityMergeReader implements IPointReader {
     }
   }
 
-  class Element {
+  static class Element {
 
     IPointReader reader;
     TimeValuePair timeValuePair;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java b/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java
index 28fcd4e..5695e3f 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java
@@ -39,7 +39,11 @@ import org.apache.iotdb.tsfile.read.reader.IBatchReader;
  */
 public class ServerTimeGenerator extends TimeGenerator {
 
-  private QueryContext context;
+  protected QueryContext context;
+
+  public ServerTimeGenerator(QueryContext context) {
+    this.context = context;
+  }
 
   /**
    * Constructor of EngineTimeGenerator.
@@ -50,7 +54,7 @@ public class ServerTimeGenerator extends TimeGenerator {
     try {
       super.constructNode(expression);
     } catch (IOException e) {
-      throw new StorageEngineException(e.getMessage());
+      throw new StorageEngineException(e);
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index 0d2c4c5..072e2cb 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -96,7 +96,9 @@ public class IoTDB implements IoTDBMBean {
     registerManager.register(UpgradeSevice.getINSTANCE());
     registerManager.register(MergeManager.getINSTANCE());
     registerManager.register(CacheHitRatioMonitor.getInstance());
-    registerManager.register(MetricsService.getInstance());
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableMetricService()) {
+      registerManager.register(MetricsService.getInstance());
+    }
     registerManager.register(StorageEngine.getInstance());
     JMXService.registerMBean(getInstance(), mbeanName);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
index 670ea32..0b0dea5 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java
@@ -43,7 +43,8 @@ public enum ServiceType {
       generateJmxName(IoTDBConstant.IOTDB_PACKAGE, "Cache Hit Ratio")),
 
   FLUSH_SERVICE("Flush ServerService",
-      generateJmxName("org.apache.iotdb.db.engine.pool", "Flush Manager"));
+      generateJmxName("org.apache.iotdb.db.engine.pool", "Flush Manager")),
+  CLUSTER_MONITOR_SERVICE("Cluster Monitor ServerService", "Cluster Monitor");
 
   private String name;
   private String jmxName;
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 2a7a614..8c5f056 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -83,6 +83,7 @@ import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithoutValueFilter;
 import org.apache.iotdb.db.tools.watermark.GroupedLSBWatermarkEncoder;
 import org.apache.iotdb.db.tools.watermark.WatermarkEncoder;
 import org.apache.iotdb.db.utils.QueryDataSetUtils;
+import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.service.rpc.thrift.ServerProperties;
 import org.apache.iotdb.service.rpc.thrift.TSBatchInsertionReq;
@@ -138,8 +139,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       "meet error while parsing SQL to physical plan: {}";
   public static Vector<SqlArgument> sqlArgumentsList = new Vector<>();
 
-  private Planner processor;
-  private IPlanExecutor executor;
+  protected Planner processor;
+  protected IPlanExecutor executor;
 
   // Record the username for every rpc connection (session).
   private Map<Long, String> sessionIdUsernameMap = new ConcurrentHashMap<>();
@@ -168,52 +169,6 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     executor = new PlanExecutor();
   }
 
-  public static TSDataType getSeriesType(String path) throws QueryProcessException {
-    switch (path.toLowerCase()) {
-      // authorization queries
-      case COLUMN_ROLE:
-      case COLUMN_USER:
-      case COLUMN_PRIVILEGE:
-      case COLUMN_STORAGE_GROUP:
-        return TSDataType.TEXT;
-      case COLUMN_TTL:
-        return TSDataType.INT64;
-      default:
-        // do nothing
-    }
-
-    if (path.contains("(") && !path.startsWith("(") && path.endsWith(")")) {
-      // aggregation
-      int leftBracketIndex = path.indexOf('(');
-      String aggrType = path.substring(0, leftBracketIndex);
-      String innerPath = path.substring(leftBracketIndex + 1, path.length() - 1);
-      switch (aggrType.toLowerCase()) {
-        case SQLConstant.MIN_TIME:
-        case SQLConstant.MAX_TIME:
-        case SQLConstant.COUNT:
-          return TSDataType.INT64;
-        case SQLConstant.LAST_VALUE:
-        case SQLConstant.FIRST_VALUE:
-        case SQLConstant.MIN_VALUE:
-        case SQLConstant.MAX_VALUE:
-          return getSeriesType(innerPath);
-        case SQLConstant.AVG:
-        case SQLConstant.SUM:
-          return TSDataType.DOUBLE;
-        default:
-          throw new QueryProcessException("aggregate does not support " + aggrType + " function.");
-      }
-    }
-    TSDataType dataType;
-    try {
-      dataType = MManager.getInstance().getSeriesType(path);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-
-    return dataType;
-  }
-
   @Override
   public TSOpenSessionResp openSession(TSOpenSessionReq req) throws TException {
     logger.info(
@@ -350,7 +305,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
   /**
    * release single operation resource
    */
-  private void releaseQueryResource(long queryId) throws StorageEngineException {
+  protected void releaseQueryResource(long queryId) throws StorageEngineException {
     // remove the corresponding Physical Plan
     queryId2DataSet.remove(queryId);
     QueryResourceManager.getInstance().endQuery(queryId);
@@ -377,6 +332,14 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return new TSStatus(statusCodeAndMessage);
   }
 
+  protected TSDataType getSeriesType(String path) throws QueryProcessException, MetadataException {
+    try {
+      return SchemaUtils.getSeriesType(path);
+    } catch (MetadataException e) {
+      throw new QueryProcessException(e);
+    }
+  }
+
   @Override
   public TSFetchMetadataResp fetchMetadata(TSFetchMetadataReq req) {
     TSStatus status;
@@ -406,7 +369,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
           status = getStatus(TSStatusCode.METADATA_ERROR, req.getType());
           break;
       }
-    } catch (QueryProcessException | MetadataException | OutOfMemoryError e) {
+    } catch (MetadataException | OutOfMemoryError | QueryProcessException e) {
       logger.error(
           String.format("Failed to fetch timeseries %s's metadata", req.getColumnPath()), e);
       status = getStatus(TSStatusCode.METADATA_ERROR, e.getMessage());
@@ -814,7 +777,11 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     }
 
     for (String column : respColumns) {
-      columnTypes.add(getSeriesType(column).toString());
+      try {
+        columnTypes.add(getSeriesType(column).toString());
+      } catch (MetadataException e) {
+        throw new QueryProcessException(e);
+      }
     }
   }
 
@@ -1047,12 +1014,16 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       throws QueryProcessException, QueryFilterOptimizationException, StorageEngineException,
       IOException, MetadataException, SQLException {
 
-    QueryContext context = new QueryContext(queryId);
+    QueryContext context = genQueryContext(queryId);
     QueryDataSet queryDataSet = executor.processQuery(physicalPlan, context);
     queryId2DataSet.put(queryId, queryDataSet);
     return queryDataSet;
   }
 
+  protected QueryContext genQueryContext(long queryId) {
+    return new QueryContext(queryId);
+  }
+
   @Override
   public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req) {
     try {
@@ -1149,7 +1120,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return resp;
   }
 
-  void handleClientExit() {
+  protected void handleClientExit() {
     Long sessionId = currSessionId.get();
     if (sessionId != null) {
       TSCloseSessionReq req = new TSCloseSessionReq(sessionId);
@@ -1435,7 +1406,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
     return null;
   }
 
-  private TSStatus executePlan(PhysicalPlan plan) {
+  protected TSStatus executePlan(PhysicalPlan plan) {
     boolean execRet;
     try {
       execRet = executeNonQuery(plan);
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
index 6c4c569..14fbdc5 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
@@ -35,6 +35,7 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.engine.merge.task.MergeTask;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java b/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java
index 780603f..262de5f 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java
@@ -118,7 +118,7 @@ public class IoTDBDataDirViewer {
   private static void printResource(String filename, PrintWriter pw) throws IOException {
     filename = filename.substring(0, filename.length() - 9);
     TsFileResource resource = new TsFileResource(SystemFileFactory.INSTANCE.getFile(filename));
-    resource.deSerialize();
+    resource.deserialize();
     // sort device strings
     SortedSet<String> keys = new TreeSet<>(resource.getStartTimeMap().keySet());
     for (String device : keys) {
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java
index d52a220..03e97d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java
@@ -54,7 +54,7 @@ public class TsFileResourcePrinter {
     filename = filename.substring(0, filename.length() - 9);
     TsFileResource resource = new TsFileResource(SystemFileFactory.INSTANCE.getFile(filename));
     System.err.println(String.format("analyzing %s ...", filename));
-    resource.deSerialize();
+    resource.deserialize();
 
     System.out.println("historicalVersions: " + resource.getHistoricalVersions());
 
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 cd011ac..98b9311 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
@@ -44,7 +44,7 @@ public class FileLoaderUtils {
       // write .resource file
       tsFileResource.serialize();
     } else {
-      tsFileResource.deSerialize();
+      tsFileResource.deserialize();
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
index c18ce58..8ae97d8 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
@@ -18,15 +18,34 @@
  */
 package org.apache.iotdb.db.utils;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_STORAGE_GROUP;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER;
+
+import java.util.Collections;
 import java.util.List;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.Schema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class SchemaUtils {
 
+  private static final Logger logger = LoggerFactory.getLogger(SchemaUtils.class);
+
   private SchemaUtils() {
   }
 
@@ -48,11 +67,75 @@ public class SchemaUtils {
    * @param schemaList the schema of the columns in this file.
    * @return a Schema contains the provided schemas.
    */
-  public static Schema getSchemaFromColumnSchema(List<MeasurementSchema> schemaList) {
+  private static Schema getSchemaFromColumnSchema(List<MeasurementSchema> schemaList) {
     Schema schema = new Schema();
     for (MeasurementSchema measurementSchema : schemaList) {
       schema.registerMeasurement(measurementSchema);
     }
     return schema;
   }
+
+  public static void registerTimeseries(MeasurementSchema schema) {
+    try {
+      logger.debug("Registering timeseries {}", schema);
+      String path = schema.getMeasurementId();
+      TSDataType dataType = schema.getType();
+      TSEncoding encoding = schema.getEncodingType();
+      CompressionType compressionType = schema.getCompressor();
+      boolean result = MManager.getInstance().createTimeseries(path, dataType, encoding,
+          compressionType, Collections.emptyMap());
+      if (result) {
+        StorageEngine.getInstance().addTimeSeries(new Path(path), dataType, encoding,
+            compressionType, Collections.emptyMap());
+      }
+    } catch (PathAlreadyExistException ignored) {
+      // ignore added timeseries
+    } catch (MetadataException | StorageEngineException e) {
+      logger.error("Cannot create timeseries {} in snapshot, ignored", schema.getMeasurementId(),
+          e);
+    }
+
+  }
+
+  public static TSDataType getSeriesType(String path)
+      throws MetadataException {
+    switch (path.toLowerCase()) {
+      // authorization queries
+      case COLUMN_ROLE:
+      case COLUMN_USER:
+      case COLUMN_PRIVILEGE:
+      case COLUMN_STORAGE_GROUP:
+        return TSDataType.TEXT;
+      case SQLConstant.RESERVED_TIME:
+      case COLUMN_TTL:
+        return TSDataType.INT64;
+      default:
+        // do nothing
+    }
+
+    if (path.contains("(") && !path.startsWith("(") && path.endsWith(")")) {
+      // aggregation
+      int leftBracketIndex = path.indexOf('(');
+      String aggrType = path.substring(0, leftBracketIndex);
+      String innerPath = path.substring(leftBracketIndex + 1, path.length() - 1);
+      switch (aggrType.toLowerCase()) {
+        case SQLConstant.MIN_TIME:
+        case SQLConstant.MAX_TIME:
+        case SQLConstant.COUNT:
+          return TSDataType.INT64;
+        case SQLConstant.LAST_VALUE:
+        case SQLConstant.FIRST_VALUE:
+        case SQLConstant.MIN_VALUE:
+        case SQLConstant.MAX_VALUE:
+          return getSeriesType(innerPath);
+        case SQLConstant.AVG:
+        case SQLConstant.SUM:
+          return TSDataType.DOUBLE;
+        default:
+          throw new MetadataException(
+              "aggregate does not support " + aggrType + " function.");
+      }
+    }
+    return MManager.getInstance().getSeriesType(path);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java b/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java
index d4058ba..fa1acc2 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java
@@ -27,5 +27,9 @@ import java.lang.annotation.Target;
 
 @Target({ElementType.METHOD, ElementType.CONSTRUCTOR})
 @Retention(RetentionPolicy.SOURCE)
+/**
+ * TestOnly implies that the method should only be used in the tests, otherwise its functionality
+ * is not guaranteed and may interfere with the normal code.
+ */
 public @interface TestOnly {
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
index 941b494..bd0c2d6 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
@@ -150,7 +150,7 @@ public class TsFileRecoverPerformer {
 
   private void recoverResourceFromFile() throws IOException {
     try {
-      tsFileResource.deSerialize();
+      tsFileResource.deserialize();
     } catch (IOException e) {
       logger.warn("Cannot deserialize TsFileResource {}, construct it using "
           + "TsFileSequenceReader", tsFileResource.getFile(), e);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
index cf4bd93..4ad0991 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
@@ -33,7 +33,6 @@ import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.merge.task.MergeTask;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.junit.After;
@@ -45,7 +44,7 @@ public class MergeLogTest extends MergeTest {
   File tempSGDir;
 
   @Before
-  public void setUp() throws IOException, WriteProcessException, MetadataException, PathException {
+  public void setUp() throws IOException, WriteProcessException, MetadataException {
     super.setUp();
     tempSGDir = new File(TestConstant.BASE_OUTPUT_PATH.concat("tempSG"));
     tempSGDir.mkdirs();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
index a4b5e7c..656a215 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
@@ -26,9 +26,8 @@ import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.merge.task.MergeTask;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
+import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
@@ -57,7 +56,7 @@ public class MergeOverLapTest extends MergeTest {
 
   @Before
   public void setUp()
-      throws IOException, WriteProcessException, MetadataException, PathException {
+      throws IOException, WriteProcessException, MetadataException {
     ptNum = 1000;
     super.setUp();
     tempSGDir = new File(TestConstant.BASE_OUTPUT_PATH.concat("tempSG"));
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
index e333c66..7b61725 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java
@@ -25,10 +25,9 @@ import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.merge.task.MergeTask;
 import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
 import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
@@ -51,7 +50,7 @@ public class MergeTaskTest extends MergeTest {
   private File tempSGDir;
 
   @Before
-  public void setUp() throws IOException, WriteProcessException, MetadataException, PathException {
+  public void setUp() throws IOException, WriteProcessException, MetadataException, MetadataException {
     super.setUp();
     tempSGDir = new File(TestConstant.BASE_OUTPUT_PATH.concat("tempSG"));
     tempSGDir.mkdirs();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
index 21a4683..028f0d0 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
@@ -34,7 +34,6 @@ import org.apache.iotdb.db.engine.cache.TsFileMetaDataCache;
 import org.apache.iotdb.db.engine.merge.manage.MergeManager;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.query.control.FileReaderManager;
@@ -71,7 +70,7 @@ abstract class MergeTest {
   private int prevMergeChunkThreshold;
 
   @Before
-  public void setUp() throws IOException, WriteProcessException, MetadataException, PathException {
+  public void setUp() throws IOException, WriteProcessException, MetadataException, MetadataException {
     MManager.getInstance().init();
     prevMergeChunkThreshold =
         IoTDBDescriptor.getInstance().getConfig().getChunkMergePointThreshold();
@@ -94,7 +93,7 @@ abstract class MergeTest {
     MergeManager.getINSTANCE().stop();
   }
 
-  private void prepareSeries() throws MetadataException, PathException {
+  private void prepareSeries() throws MetadataException, MetadataException {
     measurementSchemas = new MeasurementSchema[measurementNum];
     for (int i = 0; i < measurementNum; i++) {
       measurementSchemas[i] = new MeasurementSchema("sensor" + i, TSDataType.DOUBLE,
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
index 5be94ea..1ad0d22 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
@@ -35,19 +35,16 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.io.LocalTextModificationAccessor;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
+import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.storageGroup.StorageGroupException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-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.reader.IPointReader;
@@ -232,7 +229,7 @@ public class DeletionFileNodeTest {
 
   @Test
   public void testDeleteInOverflowFile()
-      throws StorageEngineException, QueryProcessException, IOException {
+      throws StorageEngineException, QueryProcessException {
     // insert into BufferWrite
     for (int i = 101; i <= 200; i++) {
       TSRecord record = new TSRecord(i, processorName);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
index 9e4f16a..ce79ceb 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.exception.storageGroup.StorageGroupException;
 import org.apache.iotdb.db.metadata.MManager;
@@ -39,7 +38,6 @@ import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
 import org.apache.iotdb.db.query.executor.QueryRouter;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -66,8 +64,8 @@ public class DeletionQueryTest {
   }
 
   @Before
-  public void setup() throws MetadataException,
-      PathException, IOException, StorageEngineException, StartupException, StorageGroupException {
+  public void setup() throws
+      MetadataException, StorageEngineException, StartupException {
     EnvironmentUtils.envSetUp();
 
     MManager.getInstance().setStorageGroup(processorName);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
index 7fea61d..fc8d93a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.exception.storageGroup.StorageGroupException;
 import org.apache.iotdb.db.metadata.MManager;
@@ -65,7 +64,7 @@ public class FileNodeManagerBenchmark {
   }
 
   private static void prepare()
-      throws MetadataException, PathException, IOException, StorageGroupException {
+      throws MetadataException {
     MManager manager = MManager.getInstance();
     manager.setStorageGroup(prefix);
     for (String device : devices) {
@@ -81,8 +80,8 @@ public class FileNodeManagerBenchmark {
   }
 
   public static void main(String[] args)
-      throws InterruptedException, IOException, MetadataException,
-      PathException, StorageEngineException, StorageGroupException {
+      throws InterruptedException, IOException,
+      MetadataException, StorageEngineException {
     tearDown();
     prepare();
     long startTime = System.currentTimeMillis();
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 15ce293..81fd0ea 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
@@ -37,9 +37,7 @@ import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.OutOfTTLException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.storageGroup.StorageGroupException;
 import org.apache.iotdb.db.exception.storageGroup.StorageGroupProcessorException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
@@ -76,7 +74,7 @@ public class TTLTest {
 
   @Before
   public void setUp()
-      throws MetadataException, IOException, StartupException, PathException, StorageGroupProcessorException {
+      throws MetadataException, IOException, StartupException, StorageGroupProcessorException {
     IoTDBDescriptor.getInstance().getConfig().setPartitionInterval(86400);
     EnvironmentUtils.envSetUp();
     createSchemas();
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
index a814340..5d98103 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java
@@ -408,7 +408,7 @@ public class IoTDBMultiSeriesIT {
       fail("not throw exception when unknown time series in where clause");
     } catch (SQLException e) {
       assertEquals(
-          "401: Statement format is not right: Filter has some time series don't correspond to any known time series",
+          "Statement format is not right: Filter has some time series don't correspond to any known time series",
           e.getMessage());
     }
   }
@@ -426,7 +426,7 @@ public class IoTDBMultiSeriesIT {
     } catch (SQLException e) {
       e.printStackTrace();
       assertEquals(
-          "401: Statement format is not right: Path [root.vehicle.d0.s10] does not exist",
+          "org.apache.iotdb.db.exception.metadata.PathNotExistException: Path [root.vehicle.d0.s10] does not exist",
           e.getMessage());
     }
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
index 04c9cf2..2fa9efe 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java
@@ -26,7 +26,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
-import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
@@ -98,7 +97,7 @@ public class IoTDBSequenceDataQueryIT {
     EnvironmentUtils.cleanEnv();
   }
 
-  private static void insertData() throws ClassNotFoundException, SQLException {
+  private static void insertData() throws ClassNotFoundException {
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection = DriverManager
         .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -133,7 +132,7 @@ public class IoTDBSequenceDataQueryIT {
 
       // insert data (time from 1200-1499)
       for (long time = 1200; time < 1500; time++) {
-        String sql = null;
+        String sql;
         if (time % 2 == 0) {
           sql = String
               .format("insert into root.vehicle.d0(timestamp,s0) values(%s,%s)", time, time % 17);
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
index 9d16ee0..232d9ad 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java
@@ -104,7 +104,7 @@ public class IoTDBSeriesReaderIT {
     EnvironmentUtils.cleanEnv();
   }
 
-  private static void insertData() throws ClassNotFoundException, SQLException {
+  private static void insertData() throws ClassNotFoundException {
     Class.forName(Config.JDBC_DRIVER_NAME);
     try (Connection connection = DriverManager
         .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
@@ -341,6 +341,7 @@ public class IoTDBSeriesReaderIT {
     Path path2 = new Path(Constant.d0s1);
     SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(path1,
         ValueFilter.lt(111));
+
     List<Path> pathList = new ArrayList<>();
     List<TSDataType> dataTypes = new ArrayList<>();
     pathList.add(path1);
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
index 341627a..4fd5f13 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java
@@ -142,7 +142,6 @@ public class IoTDBTtlIT {
         Statement statement = connection.createStatement()) {
       statement.execute("SET STORAGE GROUP TO root.group1");
       statement.execute("SET STORAGE GROUP TO root.group2");
-
       String result = doQuery(statement, "SHOW ALL TTL");
       assertEquals("root.group1,null\n"
           + "root.group2,null\n", result);
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java
index f6d6b67..015a094 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import org.antlr.v4.runtime.RecognitionException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.Planner;
@@ -49,7 +48,7 @@ public class ConcatOptimizerTest {
   private Planner processor;
 
   @Before
-  public void before() throws MetadataException, PathException {
+  public void before() throws MetadataException {
     processor = new Planner();
     MManager.getInstance().init();
     MManager.getInstance().setStorageGroup("root.laptop");
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index c2371c9..4a5be46 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -92,8 +92,7 @@ public class PhysicalPlanTest {
     String metadata = "create timeseries root.vehicle.d1.s2 with datatype=INT32,encoding=RLE";
     Planner processor = new Planner();
     CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata);
-    assertEquals(String.format("seriesPath: root.vehicle.d1.s2%n" + "resultDataType: INT32%n" +
-        "encoding: RLE%nnamespace type: ADD_PATH%n" + "args: "), plan.toString());
+    assertEquals("seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY", plan.toString());
   }
 
   @Test
@@ -101,8 +100,7 @@ public class PhysicalPlanTest {
     String metadata = "create timeseries root.vehicle.d1.s2 with datatype=int32,encoding=rle";
     Planner processor = new Planner();
     CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata);
-    assertEquals(String.format("seriesPath: root.vehicle.d1.s2%n" + "resultDataType: INT32%n" +
-        "encoding: RLE%nnamespace type: ADD_PATH%n" + "args: "), plan.toString());
+    assertEquals("seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY", plan.toString());
   }
 
   @Test
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java
index c52ee5b..20ebd90 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java
@@ -78,4 +78,6 @@ public class TimeValuePair implements Serializable, Comparable<TimeValuePair> {
   public int compareTo(TimeValuePair o) {
     return Long.compare(this.getTimestamp(), o.getTimestamp());
   }
+
+
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
index eb483ac..a4c316f 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java
@@ -18,16 +18,20 @@
  */
 package org.apache.iotdb.tsfile.read.common;
 
+import java.io.Serializable;
+import java.util.ArrayList;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.reader.BatchDataIterator;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.*;
-
-import java.io.Serializable;
-import java.util.ArrayList;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBinary;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBoolean;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsInt;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong;
 
 /**
  * <code>BatchData</code> is a self-defined data structure which is optimized for different type of
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
index e8cdb81..f2d82bf 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
@@ -146,4 +146,12 @@ public class GroupByFilter implements Filter, Serializable {
   public int hashCode() {
     return Objects.hash(interval, slidingStep, startTime, endTime);
   }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public long getEndTime() {
+    return endTime;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
index 1bb76d4..3767f01 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
@@ -118,4 +118,8 @@ public class In<T extends Comparable<T>> implements Filter {
   public FilterSerializeId getSerializeId() {
     return FilterSerializeId.IN;
   }
+
+  public Set<T> getValues() {
+    return values;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java
index 2bb5118..80ebf0a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java
@@ -23,29 +23,44 @@ import java.nio.ByteBuffer;
 public class Murmur128Hash {
 
   /**
-   *  get hashcode of value by seed
+   * get hashcode of value by seed
+   *
    * @param value value
    * @param seed seend
    * @return hashcode of value
    */
-  public static int hash(String value, int seed){
-    return (int) hash3_x64_128(ByteBuffer.wrap(value.getBytes()), 0, value.length(), seed);
+  public static int hash(String value, int seed) {
+    return (int) hash3_x64_128(value.getBytes(), 0, value.getBytes().length, seed);
+  }
+
+  /**
+   * get hashcode of two values by seed
+   *
+   * @param value1 the first value
+   * @param value2 the second value
+   * @param seed seend
+   * @return hashcode of value
+   */
+  public static int hash(String value1, long value2, int seed) {
+    return (int) hash3_x64_128(
+        BytesUtils.concatByteArray(value1.getBytes(), BytesUtils.longToBytes(value2)), 0,
+        value1.length() + 8, seed);
   }
 
   /**************************************
    * Methods to perform murmur 128 hash.
    **************************************/
-  private static long getBlock(ByteBuffer key, int offset, int index) {
+  private static long getBlock(byte[] key, int offset, int index) {
     int i_8 = index << 3;
     int blockOffset = offset + i_8;
-    return ((long) key.get(blockOffset) & 0xff) + (((long) key.get(blockOffset + 1) & 0xff) << 8)
+    return ((long) key[blockOffset] & 0xff) + (((long) key[blockOffset + 1] & 0xff) << 8)
         +
-        (((long) key.get(blockOffset + 2) & 0xff) << 16) + (
-        ((long) key.get(blockOffset + 3) & 0xff) << 24) +
-        (((long) key.get(blockOffset + 4) & 0xff) << 32) + (
-        ((long) key.get(blockOffset + 5) & 0xff) << 40) +
-        (((long) key.get(blockOffset + 6) & 0xff) << 48) + (
-        ((long) key.get(blockOffset + 7) & 0xff) << 56);
+        (((long) key[blockOffset + 2] & 0xff) << 16) + (
+        ((long) key[blockOffset + 3] & 0xff) << 24) +
+        (((long) key[blockOffset + 4] & 0xff) << 32) + (
+        ((long) key[blockOffset + 5] & 0xff) << 40) +
+        (((long) key[blockOffset + 6] & 0xff) << 48) + (
+        ((long) key[blockOffset + 7] & 0xff) << 56);
   }
 
   private static long rotl64(long v, int n) {
@@ -61,7 +76,7 @@ public class Murmur128Hash {
     return k;
   }
 
-  private static long hash3_x64_128(ByteBuffer key, int offset, int length, long seed) {
+  private static long hash3_x64_128(byte[] key, int offset, int length, long seed) {
     final int nblocks = length >> 4; // Process as 128-bit blocks.
     long h1 = seed;
     long h2 = seed;
@@ -95,53 +110,53 @@ public class Murmur128Hash {
     long k2 = 0;
     switch (length & 15) {
       case 15:
-        k2 ^= ((long) key.get(offset + 14)) << 48;
+        k2 ^= ((long) key[offset + 14]) << 48;
         // fallthrough
       case 14:
-        k2 ^= ((long) key.get(offset + 13)) << 40;
+        k2 ^= ((long) key[offset + 13]) << 40;
         // fallthrough
       case 13:
-        k2 ^= ((long) key.get(offset + 12)) << 32;
+        k2 ^= ((long) key[offset + 12]) << 32;
         // fallthrough
       case 12:
-        k2 ^= ((long) key.get(offset + 11)) << 24;
+        k2 ^= ((long) key[offset + 11]) << 24;
         // fallthrough
       case 11:
-        k2 ^= ((long) key.get(offset + 10)) << 16;
+        k2 ^= ((long) key[offset + 10]) << 16;
         // fallthrough
       case 10:
-        k2 ^= ((long) key.get(offset + 9)) << 8;
+        k2 ^= ((long) key[offset + 9]) << 8;
         // fallthrough
       case 9:
-        k2 ^= ((long) key.get(offset + 8));
+        k2 ^= ((long) key[offset + 8]);
         k2 *= c2;
         k2 = rotl64(k2, 33);
         k2 *= c1;
         h2 ^= k2;
         // fallthrough
       case 8:
-        k1 ^= ((long) key.get(offset + 7)) << 56;
+        k1 ^= ((long) key[offset + 7]) << 56;
         // fallthrough
       case 7:
-        k1 ^= ((long) key.get(offset + 6)) << 48;
+        k1 ^= ((long) key[offset + 6]) << 48;
         // fallthrough
       case 6:
-        k1 ^= ((long) key.get(offset + 5)) << 40;
+        k1 ^= ((long) key[offset + 5]) << 40;
         // fallthrough
       case 5:
-        k1 ^= ((long) key.get(offset + 4)) << 32;
+        k1 ^= ((long) key[offset + 4]) << 32;
         // fallthrough
       case 4:
-        k1 ^= ((long) key.get(offset + 3)) << 24;
+        k1 ^= ((long) key[offset + 3]) << 24;
         // fallthrough
       case 3:
-        k1 ^= ((long) key.get(offset + 2)) << 16;
+        k1 ^= ((long) key[offset + 2]) << 16;
         // fallthrough
       case 2:
-        k1 ^= ((long) key.get(offset + 1)) << 8;
+        k1 ^= ((long) key[offset + 1]) << 8;
         // fallthrough
       case 1:
-        k1 ^= (key.get(offset));
+        k1 ^= (key[offset]);
         k1 *= c1;
         k1 = rotl64(k1, 31);
         k1 *= c2;
@@ -164,4 +179,5 @@ public class Murmur128Hash {
     return h1 + h2;
   }
 
+
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
index 9d89621..2e305d9 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
@@ -25,6 +25,7 @@ import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.DO
 import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.FLOAT;
 import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.INTEGER;
 import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.LONG;
+import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.NULL;
 import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.STRING;
 
 import java.io.DataOutputStream;
@@ -781,7 +782,7 @@ public class ReadWriteIOUtils {
   }
 
   enum ClassSerializeId {
-    LONG, DOUBLE, INTEGER, FLOAT, BINARY, BOOLEAN, STRING
+    LONG, DOUBLE, INTEGER, FLOAT, BINARY, BOOLEAN, STRING, NULL
   }
 
   public static void writeObject(Object value, DataOutputStream outputStream) {
@@ -806,6 +807,8 @@ public class ReadWriteIOUtils {
         } else if (value instanceof Boolean) {
           outputStream.write(BOOLEAN.ordinal());
           outputStream.write(((Boolean) value) ? 1 : 0);
+        } else if (value == null) {
+          outputStream.write(NULL.ordinal());
         } else {
           outputStream.write(STRING.ordinal());
           byte[] bytes = value.toString().getBytes();
@@ -835,6 +838,8 @@ public class ReadWriteIOUtils {
         byte[] bytes = new byte[length];
         buffer.get(bytes);
         return new Binary(bytes);
+      case NULL:
+        return null;
       case STRING:
       default:
         length = buffer.getInt();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java
index b0c4a34..91d64f6 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java
@@ -318,4 +318,5 @@ public class MeasurementSchema implements Comparable<MeasurementSchema>, Seriali
     sc.addTail("]");
     return sc.toString();
   }
+
 }