You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ea...@apache.org on 2020/06/08 03:13:46 UTC

[incubator-iotdb] branch nvmlogging updated (90624d8 -> 15b4441)

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

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


    from 90624d8  fix conflicts
     new 5db7000  fix conflicts
     new cc6d7c1  fix sort bug
     new ee34ce5  fix ut bug
     new 15b4441  adjust structure

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


Summary of changes:
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   1 +
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   2 +-
 .../iotdb/db/engine/memtable/AbstractMemTable.java |   4 +-
 ...MemChunk.java => AbstractWritableMemChunk.java} |  19 +-
 .../db/engine/memtable/PrimitiveMemTable.java      |  18 --
 .../iotdb/db/engine/memtable/WritableMemChunk.java | 206 +--------------
 .../memtable/nvm}/NVMPrimitiveMemTable.java        |  69 +++---
 .../memtable/nvm/NVMWritableMemChunk.java}         |  31 +--
 .../NVMSpaceManagerException.java}                 |  12 +-
 .../nvm/{space => }/NVMSpaceMetadataManager.java   |  21 +-
 .../db/nvm/exception/NVMSpaceManagerException.java |  14 --
 .../apache/iotdb/db/nvm/executor/INVMExecutor.java |  12 -
 .../apache/iotdb/db/nvm/executor/NVMExecutor.java  | 134 ----------
 .../iotdb/db/nvm/memtable/NVMWritableMemChunk.java | 276 ---------------------
 .../org/apache/iotdb/db/nvm/metadata/Counter.java  |  18 ++
 .../apache/iotdb/db/nvm/metadata/DataTypeMemo.java |  18 ++
 .../iotdb/db/nvm/metadata/NVMSpaceMetadata.java    |  18 ++
 .../apache/iotdb/db/nvm/metadata/OffsetMemo.java   |  18 ++
 .../iotdb/db/nvm/metadata/SpaceStatusBitMap.java   |  18 ++
 .../iotdb/db/nvm/metadata/TimeValueMapper.java     |  18 ++
 .../db/nvm/metadata/TimeseriesTimeIndexMapper.java |  22 +-
 .../nvm/recover/NVMMemtableRecoverPerformer.java   |  22 +-
 .../iotdb/db/nvm/space/NVMBinaryDataSpace.java     |  63 ++++-
 .../apache/iotdb/db/nvm/space/NVMDataSpace.java    |  26 +-
 .../org/apache/iotdb/db/nvm/space/NVMSpace.java    |  18 ++
 .../apache/iotdb/db/nvm/space/NVMSpaceManager.java |  23 +-
 .../apache/iotdb/db/nvm/space/NVMStringBuffer.java |  18 ++
 .../org/apache/iotdb/db/rescon/MemTablePool.java   |   3 +-
 .../db/{nvm => }/rescon/NVMPrimitiveArrayPool.java |  34 ++-
 .../db/utils/datastructure/AbstractTVList.java     |  52 +++-
 .../iotdb/db/utils/datastructure/BinaryTVList.java |   5 -
 .../db/utils/datastructure/BooleanTVList.java      |   5 -
 .../iotdb/db/utils/datastructure/DoubleTVList.java |   5 -
 .../iotdb/db/utils/datastructure/FloatTVList.java  |   5 -
 .../iotdb/db/utils/datastructure/IntTVList.java    |   5 -
 .../iotdb/db/utils/datastructure/LongTVList.java   |   5 -
 .../db/utils/datastructure/NVMBinaryTVList.java    | 119 +++++----
 .../db/utils/datastructure/NVMBooleanTVList.java   |  63 +++--
 .../db/utils/datastructure/NVMDoubleTVList.java    |  72 +++---
 .../db/utils/datastructure/NVMFloatTVList.java     |  63 +++--
 .../iotdb/db/utils/datastructure/NVMIntTVList.java |  63 +++--
 .../db/utils/datastructure/NVMLongTVList.java      |  63 +++--
 .../iotdb/db/utils/datastructure/NVMTVList.java    |  73 +++++-
 .../iotdb/db/utils/datastructure/TVList.java       |  12 +
 .../iotdb/db/writelog/recover/LogReplayer.java     |   1 -
 .../writelog/recover/TsFileRecoverPerformer.java   |   4 +-
 .../iotdb/db/integration/IoTDBAggregationIT.java   |   8 -
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |   8 +-
 .../iotdb/db/writelog/WriteLogNodeManagerTest.java |   5 +
 .../recover/RecoverResourceFromReaderTest.java     |   8 +-
 .../db/writelog/recover/SeqTsFileRecoverTest.java  |  13 +-
 .../writelog/recover/UnseqTsFileRecoverTest.java   |   6 +
 server/src/test/resources/iotdb-engine.properties  |   3 +-
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   2 +-
 54 files changed, 772 insertions(+), 1052 deletions(-)
 copy server/src/main/java/org/apache/iotdb/db/engine/memtable/{WritableMemChunk.java => AbstractWritableMemChunk.java} (93%)
 rename server/src/main/java/org/apache/iotdb/db/{nvm/memtable => engine/memtable/nvm}/NVMPrimitiveMemTable.java (52%)
 copy server/src/main/java/org/apache/iotdb/db/{qp/logical/sys/DeleteTimeSeriesOperator.java => engine/memtable/nvm/NVMWritableMemChunk.java} (57%)
 copy server/src/main/java/org/apache/iotdb/db/exception/{MergeException.java => nvm/NVMSpaceManagerException.java} (71%)
 rename server/src/main/java/org/apache/iotdb/db/nvm/{space => }/NVMSpaceMetadataManager.java (76%)
 delete mode 100644 server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java
 delete mode 100644 server/src/main/java/org/apache/iotdb/db/nvm/executor/INVMExecutor.java
 delete mode 100644 server/src/main/java/org/apache/iotdb/db/nvm/executor/NVMExecutor.java
 delete mode 100644 server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java
 rename server/src/main/java/org/apache/iotdb/db/{nvm => }/rescon/NVMPrimitiveArrayPool.java (61%)


[incubator-iotdb] 03/04: fix ut bug

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

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

commit ee34ce56dab81313fe83b346398c37374a24e90d
Author: mdf369 <95...@qq.com>
AuthorDate: Mon Jun 8 10:44:19 2020 +0800

    fix ut bug
---
 .../src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java   |  2 +-
 .../apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java  |  2 +-
 .../org/apache/iotdb/db/writelog/WriteLogNodeManagerTest.java |  5 +++++
 .../db/writelog/recover/RecoverResourceFromReaderTest.java    |  6 ++++++
 .../iotdb/db/writelog/recover/SeqTsFileRecoverTest.java       | 11 +++++++++++
 .../iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java     |  6 ++++++
 6 files changed, 30 insertions(+), 2 deletions(-)

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 6741805..f0c3b04 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
@@ -190,7 +190,7 @@ public class IoTDBConfig {
   /**
    * Is NVM enable.
    */
-  private boolean enableNVM = true;
+  private boolean enableNVM = false;
 
   /**
    * NVM directory.
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
index 9087abd..b9e36ff 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
@@ -174,7 +174,7 @@ public class NVMBinaryTVList extends NVMTVList {
 
   @Override
   protected void setValueForSort(int arrayIndex, int elementIndex, Object value) {
-    tempValuesForSort[arrayIndex * ARRAY_SIZE + elementIndex][0] = (Binary) value;
+    tempValuesForSort[arrayIndex][elementIndex] = (Binary) value;
   }
 
   @Override
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/WriteLogNodeManagerTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/WriteLogNodeManagerTest.java
index 7125d88..4d2d6f6 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/WriteLogNodeManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/WriteLogNodeManagerTest.java
@@ -75,6 +75,11 @@ public class WriteLogNodeManagerTest {
 
   @Test
   public void testAutoSync() throws IOException, InterruptedException {
+    // This UT is for WAL, but WAL is disabled when using NVM, so just ignore.
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableNVM()) {
+      return;
+    }
+
     // this test check that nodes in a manager will sync periodically.
     int flushWalPeriod = config.getFlushWalThreshold();
     config.setForceWalPeriodInMs(10000);
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
index 1a4857d..129c397 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
@@ -27,6 +27,7 @@ import java.io.OutputStream;
 import java.util.Collections;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
@@ -172,6 +173,11 @@ public class RecoverResourceFromReaderTest {
 
   @Test
   public void testResourceRecovery() throws StorageGroupProcessorException, IOException {
+    // This UT recovers data by WAL, but WAL is disabled when using NVM, so just ignore.
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableNVM()) {
+      return;
+    }
+
     // write a broken resourceFile
     File resourceFile = FSFactoryProducer.getFSFactory()
         .getFile(resource.getFile() + TsFileResource.RESOURCE_SUFFIX);
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
index c448855..21cde4e 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
@@ -31,6 +31,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
@@ -169,6 +170,11 @@ public class SeqTsFileRecoverTest {
 
   @Test
   public void testNonLastRecovery() throws StorageGroupProcessorException, IOException {
+    // This UT recovers data by WAL, but WAL is disabled when using NVM, so just ignore.
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableNVM()) {
+      return;
+    }
+
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix,
         versionController, resource, true, false, "root.sg");
     ActiveTimeSeriesCounter.getInstance().init(storageGroup);
@@ -218,6 +224,11 @@ public class SeqTsFileRecoverTest {
 
   @Test
   public void testLastRecovery() throws StorageGroupProcessorException, IOException {
+    // This UT recovers data by WAL, but WAL is disabled when using NVM, so just ignore.
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableNVM()) {
+      return;
+    }
+
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix,
         versionController, resource, true, true, "root.sg");
     ActiveTimeSeriesCounter.getInstance().init(storageGroup);
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
index 1e34e5b..b9f84ae 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/UnseqTsFileRecoverTest.java
@@ -25,6 +25,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.adapter.ActiveTimeSeriesCounter;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
@@ -177,6 +178,11 @@ public class UnseqTsFileRecoverTest {
 
   @Test
   public void test() throws StorageGroupProcessorException, IOException {
+    // This UT recovers data by WAL, but WAL is disabled when using NVM, so just ignore.
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableNVM()) {
+      return;
+    }
+
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix,
         versionController, resource, true, false, "root.sg");
     ActiveTimeSeriesCounter.getInstance()


[incubator-iotdb] 04/04: adjust structure

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

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

commit 15b444141ad119387e90fe582f899b04d373ca50
Author: mdf369 <95...@qq.com>
AuthorDate: Mon Jun 8 11:13:17 2020 +0800

    adjust structure
---
 .../engine/memtable/AbstractWritableMemChunk.java  |  18 +++
 .../memtable/nvm}/NVMPrimitiveMemTable.java        |  20 ++-
 .../engine/memtable/nvm/NVMWritableMemChunk.java   |  37 ++++++
 .../db/exception/nvm/NVMSpaceManagerException.java |  31 +++++
 .../nvm/{space => }/NVMSpaceMetadataManager.java   |  21 +++-
 .../db/nvm/exception/NVMSpaceManagerException.java |  13 --
 .../apache/iotdb/db/nvm/executor/INVMExecutor.java |  12 --
 .../apache/iotdb/db/nvm/executor/NVMExecutor.java  | 134 ---------------------
 .../iotdb/db/nvm/memtable/NVMWritableMemChunk.java |  19 ---
 .../org/apache/iotdb/db/nvm/metadata/Counter.java  |  18 +++
 .../apache/iotdb/db/nvm/metadata/DataTypeMemo.java |  18 +++
 .../iotdb/db/nvm/metadata/NVMSpaceMetadata.java    |  18 +++
 .../apache/iotdb/db/nvm/metadata/OffsetMemo.java   |  18 +++
 .../iotdb/db/nvm/metadata/SpaceStatusBitMap.java   |  18 +++
 .../iotdb/db/nvm/metadata/TimeValueMapper.java     |  18 +++
 .../db/nvm/metadata/TimeseriesTimeIndexMapper.java |  22 +++-
 .../nvm/recover/NVMMemtableRecoverPerformer.java   |  22 +++-
 .../iotdb/db/nvm/space/NVMBinaryDataSpace.java     |  20 ++-
 .../apache/iotdb/db/nvm/space/NVMDataSpace.java    |  22 +++-
 .../org/apache/iotdb/db/nvm/space/NVMSpace.java    |  18 +++
 .../apache/iotdb/db/nvm/space/NVMSpaceManager.java |  23 +++-
 .../apache/iotdb/db/nvm/space/NVMStringBuffer.java |  18 +++
 .../org/apache/iotdb/db/rescon/MemTablePool.java   |   2 +-
 .../db/{nvm => }/rescon/NVMPrimitiveArrayPool.java |  24 +++-
 .../db/utils/datastructure/AbstractTVList.java     |  18 +++
 .../db/utils/datastructure/NVMBinaryTVList.java    |  24 +++-
 .../db/utils/datastructure/NVMBooleanTVList.java   |  20 ++-
 .../db/utils/datastructure/NVMDoubleTVList.java    |  20 ++-
 .../db/utils/datastructure/NVMFloatTVList.java     |  20 ++-
 .../iotdb/db/utils/datastructure/NVMIntTVList.java |  20 ++-
 .../db/utils/datastructure/NVMLongTVList.java      |  20 ++-
 .../iotdb/db/utils/datastructure/NVMTVList.java    |  24 +++-
 .../writelog/recover/TsFileRecoverPerformer.java   |   2 +-
 33 files changed, 545 insertions(+), 207 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java
index eb277e0..407477b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.db.utils.datastructure.AbstractTVList;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/nvm/NVMPrimitiveMemTable.java
similarity index 78%
rename from server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java
rename to server/src/main/java/org/apache/iotdb/db/engine/memtable/nvm/NVMPrimitiveMemTable.java
index 9012d4a..277a3e8 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/nvm/NVMPrimitiveMemTable.java
@@ -1,4 +1,22 @@
-package org.apache.iotdb.db.nvm.memtable;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.engine.memtable.nvm;
 
 import java.util.HashMap;
 import java.util.List;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/nvm/NVMWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/nvm/NVMWritableMemChunk.java
new file mode 100644
index 0000000..61c852a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/nvm/NVMWritableMemChunk.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.engine.memtable.nvm;
+
+import java.util.List;
+import org.apache.iotdb.db.engine.memtable.AbstractWritableMemChunk;
+import org.apache.iotdb.db.nvm.space.NVMDataSpace;
+import org.apache.iotdb.db.utils.datastructure.NVMTVList;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+public class NVMWritableMemChunk extends AbstractWritableMemChunk {
+
+  public NVMWritableMemChunk(MeasurementSchema schema, NVMTVList list) {
+    this.schema = schema;
+    this.list = list;
+  }
+
+  public void loadData(List<NVMDataSpace> timeSpaceList, List<NVMDataSpace> valueSpaceList) {
+    ((NVMTVList) list).loadData(timeSpaceList, valueSpaceList);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/nvm/NVMSpaceManagerException.java b/server/src/main/java/org/apache/iotdb/db/exception/nvm/NVMSpaceManagerException.java
new file mode 100644
index 0000000..98747fb
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/nvm/NVMSpaceManagerException.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.exception.nvm;
+
+import org.apache.iotdb.db.exception.IoTDBException;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class NVMSpaceManagerException extends IoTDBException {
+
+  private static final long serialVersionUID = 3502239072309147687L;
+
+  public NVMSpaceManagerException(String message) {
+    super(message, TSStatusCode.NVMSPACE_MANAGER_ERROR.getStatusCode());
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceMetadataManager.java b/server/src/main/java/org/apache/iotdb/db/nvm/NVMSpaceMetadataManager.java
similarity index 76%
rename from server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceMetadataManager.java
rename to server/src/main/java/org/apache/iotdb/db/nvm/NVMSpaceMetadataManager.java
index 0373433..787d90e 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceMetadataManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/NVMSpaceMetadataManager.java
@@ -1,4 +1,22 @@
-package org.apache.iotdb.db.nvm.space;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.nvm;
 
 import java.io.IOException;
 import java.util.List;
@@ -8,6 +26,7 @@ import org.apache.iotdb.db.nvm.metadata.Counter;
 import org.apache.iotdb.db.nvm.metadata.SpaceStatusBitMap;
 import org.apache.iotdb.db.nvm.metadata.TimeValueMapper;
 import org.apache.iotdb.db.nvm.metadata.TimeseriesTimeIndexMapper;
+import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 public class NVMSpaceMetadataManager {
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java b/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java
deleted file mode 100644
index 1043f46..0000000
--- a/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.iotdb.db.nvm.exception;
-
-import org.apache.iotdb.db.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
-
-public class NVMSpaceManagerException extends IoTDBException {
-
-  private static final long serialVersionUID = 3502239072309147687L;
-
-  public NVMSpaceManagerException(String message) {
-    super(message, TSStatusCode.NVMSPACE_MANAGER_ERROR.getStatusCode());
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/executor/INVMExecutor.java b/server/src/main/java/org/apache/iotdb/db/nvm/executor/INVMExecutor.java
deleted file mode 100644
index 52c8541..0000000
--- a/server/src/main/java/org/apache/iotdb/db/nvm/executor/INVMExecutor.java
+++ /dev/null
@@ -1,12 +0,0 @@
-//package org.apache.iotdb.db.nvm.executor;
-//
-//public interface INVMExecutor {
-//
-//  void flushRecordToNVM(Record record);
-//
-//  void commit(Transaction transaction);
-//
-//  void abort(Transaction transaction);
-//
-//  void recovery();
-//}
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/executor/NVMExecutor.java b/server/src/main/java/org/apache/iotdb/db/nvm/executor/NVMExecutor.java
deleted file mode 100644
index 3fa5c2d..0000000
--- a/server/src/main/java/org/apache/iotdb/db/nvm/executor/NVMExecutor.java
+++ /dev/null
@@ -1,134 +0,0 @@
-//package org.apache.iotdb.db.nvm.executor;
-//
-//import java.io.IOException;
-//import java.io.RandomAccessFile;
-//import java.nio.channels.FileChannel;
-//import java.nio.channels.FileChannel.MapMode;
-//import java.util.HashMap;
-//import java.util.HashSet;
-//import java.util.List;
-//import java.util.Map;
-//import java.util.Set;
-//import org.apache.iotdb.tsfile.utils.Pair;
-//import org.slf4j.Logger;
-//import org.slf4j.LoggerFactory;
-//
-//public class NVMExecutor implements INVMExecutor {
-//
-//  private static final Logger logger = LoggerFactory.getLogger(NVMExecutor.class);
-//
-//  // 逻辑PageID -> PCM中对应脏页及每个脏页对应的脏record
-//  private Map<Long, Pair<Long, Set<Long>>> mappingTable;
-//
-//  private final long FREE_SLOT_BITMAP_BUF_SIZE = 1000;
-//  private NVMSpace freeSlotBitmapBuf;
-//
-//  private final long ACTIVE_TX_LIST_BUF_SIZE = 1000;
-//  private NVMSpace activeTxListBuf;
-//
-//  private final long DATA_BUF_SIZE = 1000;
-//  private NVMSpace dataBuf;
-//
-//  // 未提交的事务XID -> 所有脏record
-//  private Map<Long, Set<Long>> transactionTable;
-//  // 被未提交事务修改的page PID -> 已提交版本page及对应record的已提交版本
-//  private Map<Long, Pair<Long, Set<Long>>> dirtyPageTable;
-//  // 每个page PID -> DRAM中的脏record列表
-//  private Map<Long, List<Long>> dirtyRecordTable;
-//
-//  private NVMExecutor() {
-//    initMappingTable();
-//
-//    initNVMFields();
-//
-//    transactionTable = new HashMap<>();
-//    dirtyPageTable = new HashMap<>();
-//    dirtyRecordTable = new HashMap<>();
-//  }
-//
-//  private static final NVMExecutor INSTANCE = new NVMExecutor();
-//  public static NVMExecutor getInstance() {
-//    return INSTANCE;
-//  }
-//
-//  /**
-//   * init by inverse table
-//   */
-//  private void initMappingTable() {
-//    // TODO
-//  }
-//
-//  private void initNVMFields() {
-//    FileChannel fc = null;
-//    try {
-//      fc = new RandomAccessFile(NVM_PATH, "rw").getChannel();
-//
-//      freeSlotBitmapBuf = fc.map(MapMode.READ_WRITE, FREE_SLOT_BITMAP_BUF_OFFSET, FREE_SLOT_BITMAP_BUF_OFFSET);
-//      activeTxListBuf = fc.map(MapMode.READ_WRITE, ACTIVE_TX_LIST_BUF_OFFSET, ACTIVE_TX_LIST_BUF_SIZE);
-//      dataBuf = fc.map(MapMode.READ_WRITE, DATA_BUF_OFFSET, DATA_BUF_SIZE);
-//    } catch (IOException e) {
-//      logger.error("Fail to init NVM fields at {}.", NVM_PATH, e);
-//    }
-//  }
-//
-//  @Override
-//  public void flushRecordToNVM(Record record) {
-//    /*
-//    Let T be the transaction that made the last update to t
-//    if t is T’s first dirty record to be flushed then
-//      Append T to the ActiveTxList in PCM
-//    Write t to a free space of PCM
-//    if there is a previously committed version of t in PCM then
-//      Add the previous version to the Dirty Page Table
-//    else if there is a copy of t (updated by T) in PCM then
-//      Invalidate the uncommitted copy
-//    Update the Mapping Table and Transaction Table */
-//
-//    if (isFirstToNVM(record.getXID())) {
-//      addActiveTx(record.getXID());
-//    }
-//
-//    writeRecord(record);
-//
-//    Record committedRecord = getPreviousComittedRecord(record);
-//    if (committedRecord != null) {
-//      if (!dirtyPageTable.containsKey(record.getPID())) {
-//        dirtyPageTable.put(record.getPID(), new Pair<>(committedRecord.getPID(), new HashSet<>()));
-//      }
-//      dirtyPageTable.get(record.getPID()).right.add(record.getRID());
-//    }
-//  }
-//
-//  private Record getPreviousComittedRecord(Record record) {
-//    // TODO
-//    return null;
-//  }
-//
-//  private void writeRecord(Record record) {
-//    // TODO
-//  }
-//
-//  private void addActiveTx(long XID) {
-//    // TODO
-//  }
-//
-//  private boolean isFirstToNVM(long XID) {
-//    // TODO
-//    return true;
-//  }
-//
-//  @Override
-//  public void commit(Transaction transaction) {
-//
-//  }
-//
-//  @Override
-//  public void abort(Transaction transaction) {
-//
-//  }
-//
-//  @Override
-//  public void recovery() {
-//
-//  }
-//}
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java
deleted file mode 100644
index 602a418..0000000
--- a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package org.apache.iotdb.db.nvm.memtable;
-
-import java.util.List;
-import org.apache.iotdb.db.engine.memtable.AbstractWritableMemChunk;
-import org.apache.iotdb.db.nvm.space.NVMDataSpace;
-import org.apache.iotdb.db.utils.datastructure.NVMTVList;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-public class NVMWritableMemChunk extends AbstractWritableMemChunk {
-
-  public NVMWritableMemChunk(MeasurementSchema schema, NVMTVList list) {
-    this.schema = schema;
-    this.list = list;
-  }
-
-  public void loadData(List<NVMDataSpace> timeSpaceList, List<NVMDataSpace> valueSpaceList) {
-    ((NVMTVList) list).loadData(timeSpaceList, valueSpaceList);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/Counter.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/Counter.java
index da9edb0..f539c95 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/Counter.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/Counter.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/DataTypeMemo.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/DataTypeMemo.java
index 357b63b..77d61d3 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/DataTypeMemo.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/DataTypeMemo.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/NVMSpaceMetadata.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/NVMSpaceMetadata.java
index 9541bdc..119616f 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/NVMSpaceMetadata.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/NVMSpaceMetadata.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/OffsetMemo.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/OffsetMemo.java
index 114e6e4..8a85c31 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/OffsetMemo.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/OffsetMemo.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/SpaceStatusBitMap.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/SpaceStatusBitMap.java
index aae6ac0..e71d335 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/SpaceStatusBitMap.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/SpaceStatusBitMap.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeValueMapper.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeValueMapper.java
index 03dc93b..b337435 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeValueMapper.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeValueMapper.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeseriesTimeIndexMapper.java b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeseriesTimeIndexMapper.java
index 2a6d7d9..04f6395 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeseriesTimeIndexMapper.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/metadata/TimeseriesTimeIndexMapper.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.metadata;
 
 import java.io.IOException;
@@ -6,8 +24,8 @@ import org.apache.iotdb.db.nvm.space.NVMStringBuffer;
 
 public class TimeseriesTimeIndexMapper extends NVMSpaceMetadata {
 
-  // TODO
-  private final long STRING_SPACE_SIZE_MAX = 1000;
+  // TODO may set a better value
+  private final long STRING_SPACE_SIZE_MAX = 1024;
 
   private NVMStringBuffer sgIdBuffer;
   private NVMStringBuffer deviceIdBuffer;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/recover/NVMMemtableRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/nvm/recover/NVMMemtableRecoverPerformer.java
index bbb3cd5..3107cfe 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/recover/NVMMemtableRecoverPerformer.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/recover/NVMMemtableRecoverPerformer.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.recover;
 
 import java.io.IOException;
@@ -8,10 +26,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.nvm.memtable.NVMPrimitiveMemTable;
+import org.apache.iotdb.db.engine.memtable.nvm.NVMPrimitiveMemTable;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMSpaceManager;
-import org.apache.iotdb.db.nvm.space.NVMSpaceMetadataManager;
+import org.apache.iotdb.db.nvm.NVMSpaceMetadataManager;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java
index 3e3932f..37bcc44 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java
@@ -1,7 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.space;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
-
 import java.nio.ByteBuffer;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java
index d299b32..e166a12 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.space;
 
 import java.nio.ByteBuffer;
@@ -96,7 +114,7 @@ public class NVMDataSpace extends NVMSpace {
         byteBuffer.putDouble(index, (double) object);
         break;
       case TEXT:
-        // TODO
+        // Override in class NVMBinaryDataSpace, no need to implement here
         break;
     }
   }
@@ -147,7 +165,7 @@ public class NVMDataSpace extends NVMSpace {
         }
         return doubleArray;
       case TEXT:
-        // TODO
+        // Override in class NVMBinaryDataSpace, no need to implement here
         break;
     }
     return null;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpace.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpace.java
index 5d27e95..51bc2be 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpace.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpace.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.space;
 
 import java.nio.ByteBuffer;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceManager.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceManager.java
index b51c073..e4321ae 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMSpaceManager.java
@@ -1,6 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.space;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import java.io.File;
 import java.io.IOException;
@@ -10,7 +28,8 @@ import java.nio.channels.FileChannel.MapMode;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.nvm.exception.NVMSpaceManagerException;
+import org.apache.iotdb.db.exception.nvm.NVMSpaceManagerException;
+import org.apache.iotdb.db.nvm.NVMSpaceMetadataManager;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMStringBuffer.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMStringBuffer.java
index 8ab06b2..c8e7519 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMStringBuffer.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMStringBuffer.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.nvm.space;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java b/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
index 23f2a90..ca6fbfe 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
-import org.apache.iotdb.db.nvm.memtable.NVMPrimitiveMemTable;
+import org.apache.iotdb.db.engine.memtable.nvm.NVMPrimitiveMemTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java b/server/src/main/java/org/apache/iotdb/db/rescon/NVMPrimitiveArrayPool.java
similarity index 74%
rename from server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java
rename to server/src/main/java/org/apache/iotdb/db/rescon/NVMPrimitiveArrayPool.java
index 81080c2..ce320cd 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/NVMPrimitiveArrayPool.java
@@ -1,14 +1,30 @@
-package org.apache.iotdb.db.nvm.rescon;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.rescon;
 
 import java.util.ArrayDeque;
 import java.util.EnumMap;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.nvm.exception.NVMSpaceManagerException;
+import org.apache.iotdb.db.exception.nvm.NVMSpaceManagerException;
 import org.apache.iotdb.db.nvm.space.NVMBinaryDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMSpaceManager;
-import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
-import org.apache.iotdb.db.utils.datastructure.NVMBinaryTVList;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 public class NVMPrimitiveArrayPool {
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
index 4e84e8e..a996047 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
index b9e36ff..be68b86 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
@@ -1,11 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
-import org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool;
 import org.apache.iotdb.db.nvm.space.NVMBinaryDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
-import org.apache.iotdb.db.nvm.space.NVMSpaceMetadataManager;
+import org.apache.iotdb.db.nvm.NVMSpaceMetadataManager;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
index f6e7902..7622a31 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
@@ -1,6 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
index 6e12391..eadc700 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
@@ -1,6 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
index fa7b517..89bf955 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
@@ -1,6 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
index 67b7e77..24158e7 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
@@ -1,6 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
index 774890a..4922d4f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
@@ -1,6 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
index 37eea2f..722d086 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
@@ -1,12 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.iotdb.db.utils.datastructure;
 
-import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
+import static org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool;
+import org.apache.iotdb.db.rescon.NVMPrimitiveArrayPool;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
-import org.apache.iotdb.db.nvm.space.NVMSpaceMetadataManager;
+import org.apache.iotdb.db.nvm.NVMSpaceMetadataManager;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
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 425d0d3..b1bea5b 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
@@ -37,7 +37,7 @@ import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.version.VersionController;
 import org.apache.iotdb.db.exception.StorageGroupProcessorException;
-import org.apache.iotdb.db.nvm.memtable.NVMPrimitiveMemTable;
+import org.apache.iotdb.db.engine.memtable.nvm.NVMPrimitiveMemTable;
 import org.apache.iotdb.db.nvm.recover.NVMMemtableRecoverPerformer;
 import org.apache.iotdb.db.utils.FileLoaderUtils;
 import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;


[incubator-iotdb] 02/04: fix sort bug

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

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

commit cc6d7c15115f1543ddbaa9ab8e8efb8b9461fceb
Author: mdf369 <95...@qq.com>
AuthorDate: Sat Jun 6 01:27:03 2020 +0800

    fix sort bug
---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java     |  2 +-
 .../iotdb/db/engine/flush/MemTableFlushTask.java       |  5 -----
 .../iotdb/db/query/executor/AggregationExecutor.java   |  1 -
 .../iotdb/db/query/reader/series/SeriesReader.java     |  1 -
 .../iotdb/db/utils/datastructure/AbstractTVList.java   | 11 ++++-------
 .../iotdb/db/utils/datastructure/NVMBinaryTVList.java  |  9 ++++-----
 .../iotdb/db/utils/datastructure/NVMBooleanTVList.java |  5 +++--
 .../iotdb/db/utils/datastructure/NVMDoubleTVList.java  |  5 +++--
 .../iotdb/db/utils/datastructure/NVMFloatTVList.java   |  5 +++--
 .../iotdb/db/utils/datastructure/NVMIntTVList.java     |  5 +++--
 .../iotdb/db/utils/datastructure/NVMLongTVList.java    |  5 +++--
 .../apache/iotdb/db/utils/datastructure/NVMTVList.java | 18 ++++++++++++++++--
 .../apache/iotdb/db/utils/datastructure/TVList.java    | 12 ++++++++++++
 .../org/apache/iotdb/tsfile/read/common/BatchData.java |  2 +-
 .../iotdb/tsfile/read/reader/chunk/ChunkReader.java    |  1 -
 15 files changed, 53 insertions(+), 34 deletions(-)

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 f0c3b04..6741805 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
@@ -190,7 +190,7 @@ public class IoTDBConfig {
   /**
    * Is NVM enable.
    */
-  private boolean enableNVM = false;
+  private boolean enableNVM = true;
 
   /**
    * NVM directory.
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
index 8c9983c..77ec37c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
@@ -80,11 +80,6 @@ public class MemTableFlushTask {
         IWritableMemChunk series = memTable.getMemTableMap().get(deviceId).get(measurementId);
         MeasurementSchema desc = series.getSchema();
         AbstractTVList tvList = series.getSortedTVList();
-        System.out.println("flush size:" + tvList.size());
-        for (int i = 0; i < tvList.size(); i++) {
-          System.out.print("<" + tvList.getTime(i) + ":" + tvList.getInt(i) + "> ");
-        }
-        System.out.println();
         sortTime += System.currentTimeMillis() - startTime;
         encodingTaskQueue.add(new Pair<>(tvList, desc));
         // register active time series to the ActiveTimeSeriesCounter
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 fae41c4..04e80f2 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
@@ -231,7 +231,6 @@ public class AggregationExecutor {
         continue;
       }
       BatchData nextOverlappedPageData = seriesReader.nextPage();
-      System.out.println("read:" + nextOverlappedPageData.count);
       for (int i = 0; i < aggregateResultList.size(); i++) {
         if (!isCalculatedArray[i]) {
           AggregateResult aggregateResult = aggregateResultList.get(i);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
index b1e2eb6..816be29 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
@@ -477,7 +477,6 @@ class SeriesReader {
            * get current first point in mergeReader, this maybe overlapped latter
            */
           TimeValuePair timeValuePair = mergeReader.currentTimeValuePair();
-          System.out.println("read:" + timeValuePair);
 
           if (timeValuePair.getTimestamp() > currentPageEndTime) {
             break;
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
index d8728f3..4e84e8e 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
@@ -136,6 +136,8 @@ public abstract class AbstractTVList {
 
   protected abstract void set(int src, int dest);
 
+  protected abstract void setForSort(int src, int dest);
+
   protected abstract void setFromSorted(int src, int dest);
 
   protected abstract void setToSorted(int src, int dest);
@@ -234,11 +236,7 @@ public abstract class AbstractTVList {
     this.timeOffset = timeOffset;
   }
 
-  protected int compare(int idx1, int idx2) {
-    long t1 = getTime(idx1);
-    long t2 = getTime(idx2);
-    return Long.compare(t1, t2);
-  }
+  protected abstract int compare(int idx1, int idx2);
 
   protected abstract void saveAsPivot(int pos);
 
@@ -283,8 +281,7 @@ public abstract class AbstractTVList {
        */
       int n = start - left;  // The number of elements to move
       for (int i = n; i >= 1; i--) {
-        set(left + i - 1, left + i);
-        setToSorted(left + i - 1, left + i);
+        setForSort(left + i - 1, left + i);
       }
       setPivotTo(left);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
index 88c37fc..9087abd 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
@@ -2,11 +2,9 @@ package org.apache.iotdb.db.utils.datastructure;
 
 import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
-import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool;
 import org.apache.iotdb.db.nvm.space.NVMBinaryDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
-import org.apache.iotdb.db.nvm.space.NVMSpaceManager;
 import org.apache.iotdb.db.nvm.space.NVMSpaceMetadataManager;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -92,7 +90,7 @@ public class NVMBinaryTVList extends NVMTVList {
   }
 
   @Override
-  protected void initTempArrays() {
+  protected void initSortedAndTempArrays() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
       sortedTimestamps = (long[][]) PrimitiveArrayPool
           .getInstance().getDataListsByType(TSDataType.INT64, size);
@@ -163,7 +161,8 @@ public class NVMBinaryTVList extends NVMTVList {
 
   @Override
   protected void setFromSorted(int src, int dest) {
-    setForSort(dest, sortedTimestamps[src/ARRAY_SIZE][src%ARRAY_SIZE], sortedValues[src/ARRAY_SIZE][src%ARRAY_SIZE]);
+    setForSort(dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
   }
 
   @Override
@@ -175,7 +174,7 @@ public class NVMBinaryTVList extends NVMTVList {
 
   @Override
   protected void setValueForSort(int arrayIndex, int elementIndex, Object value) {
-    tempValuesForSort[arrayIndex][elementIndex] = (Binary) value;
+    tempValuesForSort[arrayIndex * ARRAY_SIZE + elementIndex][0] = (Binary) value;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
index 726fb1e..f6e7902 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
@@ -47,7 +47,7 @@ public class NVMBooleanTVList extends NVMTVList {
   }
 
   @Override
-  protected void initTempArrays() {
+  protected void initSortedAndTempArrays() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
       sortedTimestamps = (long[][]) PrimitiveArrayPool
           .getInstance().getDataListsByType(TSDataType.INT64, size);
@@ -118,7 +118,8 @@ public class NVMBooleanTVList extends NVMTVList {
 
   @Override
   protected void setFromSorted(int src, int dest) {
-    setForSort(dest, sortedTimestamps[src/ARRAY_SIZE][src%ARRAY_SIZE], sortedValues[src/ARRAY_SIZE][src%ARRAY_SIZE]);
+    setForSort(dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
index c8bdee4..6e12391 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
@@ -48,7 +48,7 @@ public class NVMDoubleTVList extends NVMTVList {
   }
 
   @Override
-  protected void initTempArrays() {
+  protected void initSortedAndTempArrays() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
       sortedTimestamps = (long[][]) PrimitiveArrayPool
           .getInstance().getDataListsByType(TSDataType.INT64, size);
@@ -119,7 +119,8 @@ public class NVMDoubleTVList extends NVMTVList {
 
   @Override
   protected void setFromSorted(int src, int dest) {
-    setForSort(dest, sortedTimestamps[src/ARRAY_SIZE][src%ARRAY_SIZE], sortedValues[src/ARRAY_SIZE][src%ARRAY_SIZE]);
+    setForSort(dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
index d72e05a..fa7b517 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
@@ -47,7 +47,7 @@ public class NVMFloatTVList extends NVMTVList {
   }
 
   @Override
-  protected void initTempArrays() {
+  protected void initSortedAndTempArrays() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
       sortedTimestamps = (long[][]) PrimitiveArrayPool
           .getInstance().getDataListsByType(TSDataType.INT64, size);
@@ -118,7 +118,8 @@ public class NVMFloatTVList extends NVMTVList {
 
   @Override
   protected void setFromSorted(int src, int dest) {
-    setForSort(dest, sortedTimestamps[src/ARRAY_SIZE][src%ARRAY_SIZE], sortedValues[src/ARRAY_SIZE][src%ARRAY_SIZE]);
+    setForSort(dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
index 13e5e0a..67b7e77 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
@@ -47,7 +47,7 @@ public class NVMIntTVList extends NVMTVList {
   }
 
   @Override
-  protected void initTempArrays() {
+  protected void initSortedAndTempArrays() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
       sortedTimestamps = (long[][]) PrimitiveArrayPool
           .getInstance().getDataListsByType(TSDataType.INT64, size);
@@ -118,7 +118,8 @@ public class NVMIntTVList extends NVMTVList {
 
   @Override
   protected void setFromSorted(int src, int dest) {
-    setForSort(dest, sortedTimestamps[src/ARRAY_SIZE][src%ARRAY_SIZE], sortedValues[src/ARRAY_SIZE][src%ARRAY_SIZE]);
+    setForSort(dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
index 98823be..774890a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
@@ -47,7 +47,7 @@ public class NVMLongTVList extends NVMTVList {
   }
 
   @Override
-  protected void initTempArrays() {
+  protected void initSortedAndTempArrays() {
     if (sortedTimestamps == null || sortedTimestamps.length < size) {
       sortedTimestamps = (long[][]) PrimitiveArrayPool
           .getInstance().getDataListsByType(TSDataType.INT64, size);
@@ -118,7 +118,8 @@ public class NVMLongTVList extends NVMTVList {
 
   @Override
   protected void setFromSorted(int src, int dest) {
-    setForSort(dest, sortedTimestamps[src/ARRAY_SIZE][src%ARRAY_SIZE], sortedValues[src/ARRAY_SIZE][src%ARRAY_SIZE]);
+    setForSort(dest, sortedTimestamps[src / ARRAY_SIZE][src % ARRAY_SIZE],
+        sortedValues[src / ARRAY_SIZE][src % ARRAY_SIZE]);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
index 6e4d2d1..37eea2f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
@@ -241,7 +241,7 @@ public abstract class NVMTVList extends AbstractTVList {
 
   @Override
   public void sort() {
-    initTempArrays();
+    initSortedAndTempArrays();
 
     copyTVToTempArrays();
 
@@ -255,7 +255,7 @@ public abstract class NVMTVList extends AbstractTVList {
     sorted = true;
   }
 
-  protected abstract void initTempArrays();
+  protected abstract void initSortedAndTempArrays();
 
   protected abstract void copyTVToTempArrays();
 
@@ -296,6 +296,13 @@ public abstract class NVMTVList extends AbstractTVList {
   }
 
   @Override
+  protected void setForSort(int src, int dest) {
+    long srcT = getTimeForSort(src);
+    Object srcV = getValueForSort(src);
+    setForSort(dest, srcT, srcV);
+  }
+
+  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     if (index >= size) {
       throw new ArrayIndexOutOfBoundsException(index);
@@ -309,6 +316,13 @@ public abstract class NVMTVList extends AbstractTVList {
   protected abstract void setValueForSort(int arrayIndex, int elementIndex, Object value);
 
   @Override
+  protected int compare(int idx1, int idx2) {
+    long t1 = getTimeForSort(idx1);
+    long t2 = getTimeForSort(idx2);
+    return Long.compare(t1, t2);
+  }
+
+  @Override
   protected void saveAsPivot(int pos) {
     pivotTime = getTimeForSort(pos);
     pivotValue = getValueForSort(pos);
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
index f9a3df7..4b7cfd3 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
@@ -108,6 +108,18 @@ public abstract class TVList extends AbstractTVList {
   }
 
   @Override
+  protected int compare(int idx1, int idx2) {
+    long t1 = getTime(idx1);
+    long t2 = getTime(idx2);
+    return Long.compare(t1, t2);
+  }
+
+  @Override
+  protected void setForSort(int src, int dest) {
+    set(src, dest);
+  }
+
+  @Override
   protected void clearSortedTime() {
     if (sortedTimestamps != null) {
       for (long[] dataArray : sortedTimestamps) {
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 da9dd83..a893c1b 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
@@ -68,7 +68,7 @@ public class BatchData implements Serializable {
   private int writeCurArrayIndex;
 
   // the insert timestamp number of timeRet
-  public int count;
+  private int count;
 
 
   private List<long[]> timeRet;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
index 4289952..aa334d8 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
@@ -93,7 +93,6 @@ public class ChunkReader implements IChunkReader {
       // deserialize a PageHeader from chunkDataBuffer
       PageHeader pageHeader = isFromOldTsFile ? HeaderUtils.deserializePageHeaderV1(chunkDataBuffer, chunkHeader.getDataType()) :
           PageHeader.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType());
-      System.out.println("statistics:" + pageHeader.getStatistics());
       // if the current page satisfies
       if (pageSatisfied(pageHeader)) {
         pageReaderList.add(constructPageReaderForNextPage(pageHeader));


[incubator-iotdb] 01/04: fix conflicts

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

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

commit 5db70006889d34445ec387857853f08cd7cab9c2
Author: mdf369 <95...@qq.com>
AuthorDate: Sat Jun 6 00:16:55 2020 +0800

    fix conflicts
---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   1 +
 .../org/apache/iotdb/db/engine/StorageEngine.java  |   2 +-
 .../iotdb/db/engine/flush/MemTableFlushTask.java   |   5 +
 .../iotdb/db/engine/memtable/AbstractMemTable.java |   4 +-
 ...MemChunk.java => AbstractWritableMemChunk.java} |  35 +--
 .../db/engine/memtable/PrimitiveMemTable.java      |  18 --
 .../iotdb/db/engine/memtable/WritableMemChunk.java | 206 +---------------
 .../db/nvm/exception/NVMSpaceManagerException.java |   7 +-
 .../db/nvm/memtable/NVMPrimitiveMemTable.java      |  49 ++--
 .../iotdb/db/nvm/memtable/NVMWritableMemChunk.java | 269 +--------------------
 .../iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java |  14 +-
 .../iotdb/db/nvm/space/NVMBinaryDataSpace.java     |  43 +++-
 .../apache/iotdb/db/nvm/space/NVMDataSpace.java    |   4 +-
 .../db/query/executor/AggregationExecutor.java     |   1 +
 .../iotdb/db/query/reader/series/SeriesReader.java |   1 +
 .../org/apache/iotdb/db/rescon/MemTablePool.java   |   1 -
 .../db/utils/datastructure/AbstractTVList.java     |  25 ++
 .../iotdb/db/utils/datastructure/BinaryTVList.java |   5 -
 .../db/utils/datastructure/BooleanTVList.java      |   5 -
 .../iotdb/db/utils/datastructure/DoubleTVList.java |   5 -
 .../iotdb/db/utils/datastructure/FloatTVList.java  |   5 -
 .../iotdb/db/utils/datastructure/IntTVList.java    |   5 -
 .../iotdb/db/utils/datastructure/LongTVList.java   |   5 -
 .../db/utils/datastructure/NVMBinaryTVList.java    |  88 +++----
 .../db/utils/datastructure/NVMBooleanTVList.java   |  38 +--
 .../db/utils/datastructure/NVMDoubleTVList.java    |  47 +---
 .../db/utils/datastructure/NVMFloatTVList.java     |  38 +--
 .../iotdb/db/utils/datastructure/NVMIntTVList.java |  38 +--
 .../db/utils/datastructure/NVMLongTVList.java      |  38 +--
 .../iotdb/db/utils/datastructure/NVMTVList.java    |  31 ++-
 .../iotdb/db/writelog/recover/LogReplayer.java     |   1 -
 .../writelog/recover/TsFileRecoverPerformer.java   |   2 +-
 .../iotdb/db/integration/IoTDBAggregationIT.java   |   8 -
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |   8 +-
 .../recover/RecoverResourceFromReaderTest.java     |   2 +-
 .../db/writelog/recover/SeqTsFileRecoverTest.java  |   2 +-
 server/src/test/resources/iotdb-engine.properties  |   3 +-
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   2 +-
 .../apache/iotdb/tsfile/read/common/BatchData.java |   2 +-
 .../tsfile/read/reader/chunk/ChunkReader.java      |   1 +
 40 files changed, 238 insertions(+), 826 deletions(-)

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 29abfb5..f0c3b04 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
@@ -646,6 +646,7 @@ public class IoTDBConfig {
     schemaDir = addHomeDir(schemaDir);
     syncDir = addHomeDir(syncDir);
     walFolder = addHomeDir(walFolder);
+    nvmDir = addHomeDir(nvmDir);
 
     if (TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs().equals(FSType.HDFS)) {
       String hdfsDir = getHdfsDir();
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 ba10dbb..65f99d4 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
@@ -157,7 +157,7 @@ public class StorageEngine implements IService {
     /*
      * recover all storage group processors.
      */
-    List<StorageGroupMNode> sgNodes = MManager.getInstance().getAllStorageGroups();
+    List<StorageGroupMNode> sgNodes = MManager.getInstance().getAllStorageGroupNodes();
     List<Future> futures = new ArrayList<>();
     for (StorageGroupMNode storageGroup : sgNodes) {
       futures.add(recoveryThreadPool.submit((Callable<Void>) () -> {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
index 77ec37c..8c9983c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java
@@ -80,6 +80,11 @@ public class MemTableFlushTask {
         IWritableMemChunk series = memTable.getMemTableMap().get(deviceId).get(measurementId);
         MeasurementSchema desc = series.getSchema();
         AbstractTVList tvList = series.getSortedTVList();
+        System.out.println("flush size:" + tvList.size());
+        for (int i = 0; i < tvList.size(); i++) {
+          System.out.print("<" + tvList.getTime(i) + ":" + tvList.getInt(i) + "> ");
+        }
+        System.out.println();
         sortTime += System.currentTimeMillis() - startTime;
         encodingTaskQueue.add(new Pair<>(tvList, desc));
         // register active time series to the ActiveTimeSeriesCounter
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index a2bb763..f7d1183 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -144,10 +144,12 @@ public abstract class AbstractMemTable implements IMemTable {
   }
 
 
+  @Override
   public int getSeriesNumber() {
     return seriesNumber;
   }
 
+  @Override
   public long getTotalPointsNum() {
     return totalPointsNum;
   }
@@ -197,7 +199,7 @@ public abstract class AbstractMemTable implements IMemTable {
     }
     long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
     IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
-    TVList chunkCopy = memChunk.getTVList().clone();
+    TVList chunkCopy = (TVList) memChunk.getTVList().clone();
 
     chunkCopy.setTimeOffset(undeletedTime);
     return new ReadOnlyMemChunk(measurement, dataType, encoding, chunkCopy, props, getVersion());
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java
similarity index 81%
copy from server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
copy to server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java
index 4e115b6..eb277e0 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractWritableMemChunk.java
@@ -1,38 +1,16 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
 package org.apache.iotdb.db.engine.memtable;
 
+import org.apache.iotdb.db.utils.datastructure.AbstractTVList;
 import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
-public class WritableMemChunk implements IWritableMemChunk {
+public abstract class AbstractWritableMemChunk implements IWritableMemChunk {
 
-  private MeasurementSchema schema;
-  private TVList list;
-
-  public WritableMemChunk(MeasurementSchema schema, TVList list) {
-    this.schema = schema;
-    this.list = list;
-  }
+  protected MeasurementSchema schema;
+  protected AbstractTVList list;
 
   @Override
   public void write(long insertTime, Object objectValue) {
@@ -92,7 +70,6 @@ public class WritableMemChunk implements IWritableMemChunk {
     }
   }
 
-
   @Override
   public void putLong(long t, long v) {
     list.putLong(t, v);
@@ -184,13 +161,13 @@ public class WritableMemChunk implements IWritableMemChunk {
   }
 
   @Override
-  public synchronized TVList getSortedTVList() {
+  public synchronized AbstractTVList getSortedTVList() {
     list.sort();
     return list;
   }
 
   @Override
-  public TVList getTVList() {
+  public AbstractTVList getTVList() {
     return list;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
index 52bda30..779d89f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java
@@ -22,7 +22,6 @@ package org.apache.iotdb.db.engine.memtable;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.rescon.TVListAllocator;
 import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
@@ -62,21 +61,4 @@ public class PrimitiveMemTable extends AbstractMemTable {
   public boolean equals(Object obj) {
     return this == obj;
   }
-
-  @Override
-  public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType dataType,
-      Map<String, String> props, long timeLowerBound) {
-    TimeValuePairSorter sorter;
-    if (!checkPath(deviceId, measurement)) {
-      return null;
-    } else {
-      long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
-      IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
-      IWritableMemChunk chunkCopy = new WritableMemChunk(dataType,
-          (TVList) memChunk.getTVList().clone());
-      chunkCopy.setTimeOffset(undeletedTime);
-      sorter = chunkCopy;
-    }
-    return new ReadOnlyMemChunk(dataType, sorter, props);
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
index 4e115b6..71422ec 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
@@ -18,219 +18,17 @@
  */
 package org.apache.iotdb.db.engine.memtable;
 
+import org.apache.iotdb.db.utils.datastructure.AbstractTVList;
 import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
-public class WritableMemChunk implements IWritableMemChunk {
-
-  private MeasurementSchema schema;
-  private TVList list;
+public class WritableMemChunk extends AbstractWritableMemChunk {
 
   public WritableMemChunk(MeasurementSchema schema, TVList list) {
     this.schema = schema;
     this.list = list;
   }
-
-  @Override
-  public void write(long insertTime, Object objectValue) {
-    switch (schema.getType()) {
-      case BOOLEAN:
-        putBoolean(insertTime, (boolean) objectValue);
-        break;
-      case INT32:
-        putInt(insertTime, (int) objectValue);
-        break;
-      case INT64:
-        putLong(insertTime, (long) objectValue);
-        break;
-      case FLOAT:
-        putFloat(insertTime, (float) objectValue);
-        break;
-      case DOUBLE:
-        putDouble(insertTime, (double) objectValue);
-        break;
-      case TEXT:
-        putBinary(insertTime, (Binary) objectValue);
-        break;
-      default:
-        throw new UnSupportedDataTypeException("Unsupported data type:" + schema.getType());
-    }
-  }
-
-  @Override
-  public void write(long[] times, Object valueList, TSDataType dataType, int start, int end) {
-    switch (dataType) {
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) valueList;
-        putBooleans(times, boolValues, start, end);
-        break;
-      case INT32:
-        int[] intValues = (int[]) valueList;
-        putInts(times, intValues, start, end);
-        break;
-      case INT64:
-        long[] longValues = (long[]) valueList;
-        putLongs(times, longValues, start, end);
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) valueList;
-        putFloats(times, floatValues, start, end);
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) valueList;
-        putDoubles(times, doubleValues, start, end);
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) valueList;
-        putBinaries(times, binaryValues, start, end);
-        break;
-      default:
-        throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
-    }
-  }
-
-
-  @Override
-  public void putLong(long t, long v) {
-    list.putLong(t, v);
-  }
-
-  @Override
-  public void putInt(long t, int v) {
-    list.putInt(t, v);
-  }
-
-  @Override
-  public void putFloat(long t, float v) {
-    list.putFloat(t, v);
-  }
-
-  @Override
-  public void putDouble(long t, double v) {
-    list.putDouble(t, v);
-  }
-
-  @Override
-  public void putBinary(long t, Binary v) {
-    list.putBinary(t, v);
-  }
-
-  @Override
-  public void putBoolean(long t, boolean v) {
-    list.putBoolean(t, v);
-  }
-
-  @Override
-  public void putLongs(long[] t, long[] v) {
-    list.putLongs(t, v);
-  }
-
-  @Override
-  public void putInts(long[] t, int[] v) {
-    list.putInts(t, v);
-  }
-
-  @Override
-  public void putFloats(long[] t, float[] v) {
-    list.putFloats(t, v);
-  }
-
-  @Override
-  public void putDoubles(long[] t, double[] v) {
-    list.putDoubles(t, v);
-  }
-
-  @Override
-  public void putBinaries(long[] t, Binary[] v) {
-    list.putBinaries(t, v);
-  }
-
-  @Override
-  public void putBooleans(long[] t, boolean[] v) {
-    list.putBooleans(t, v);
-  }
-
-  @Override
-  public void putLongs(long[] t, long[] v, int start, int end) {
-    list.putLongs(t, v, start, end);
-  }
-
-  @Override
-  public void putInts(long[] t, int[] v, int start, int end) {
-    list.putInts(t, v, start, end);
-  }
-
-  @Override
-  public void putFloats(long[] t, float[] v, int start, int end) {
-    list.putFloats(t, v, start, end);
-  }
-
-  @Override
-  public void putDoubles(long[] t, double[] v, int start, int end) {
-    list.putDoubles(t, v, start, end);
-  }
-
-  @Override
-  public void putBinaries(long[] t, Binary[] v, int start, int end) {
-    list.putBinaries(t, v, start, end);
-  }
-
-  @Override
-  public void putBooleans(long[] t, boolean[] v, int start, int end) {
-    list.putBooleans(t, v, start, end);
-  }
-
-  @Override
-  public synchronized TVList getSortedTVList() {
-    list.sort();
-    return list;
-  }
-
-  @Override
-  public TVList getTVList() {
-    return list;
-  }
-
-  @Override
-  public long count() {
-    return list.size();
-  }
-
-  @Override
-  public MeasurementSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public void setTimeOffset(long offset) {
-    list.setTimeOffset(offset);
-  }
-
-  @Override
-  public long getMinTime() {
-    return list.getMinTime();
-  }
-
-  @Override
-  public int delete(long upperBound) {
-    return list.delete(upperBound);
-  }
-
-  @Override
-  public String toString() {
-    int size = getSortedTVList().size();
-    StringBuilder out = new StringBuilder("MemChunk Size: " + size + System.lineSeparator());
-    if (size != 0) {
-      out.append("Data type:").append(schema.getType()).append(System.lineSeparator());
-      out.append("First point:").append(getSortedTVList().getTimeValuePair(0))
-          .append(System.lineSeparator());
-      out.append("Last point:").append(getSortedTVList().getTimeValuePair(size - 1))
-          .append(System.lineSeparator());
-      ;
-    }
-    return out.toString();
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java b/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java
index 891acc0..1043f46 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/exception/NVMSpaceManagerException.java
@@ -1,14 +1,13 @@
 package org.apache.iotdb.db.nvm.exception;
 
-import org.apache.iotdb.db.exception.ProcessException;
+import org.apache.iotdb.db.exception.IoTDBException;
 import org.apache.iotdb.rpc.TSStatusCode;
 
-public class NVMSpaceManagerException extends ProcessException {
+public class NVMSpaceManagerException extends IoTDBException {
 
   private static final long serialVersionUID = 3502239072309147687L;
 
   public NVMSpaceManagerException(String message) {
-    super(message);
-    errorCode = TSStatusCode.NVMSPACE_MANAGER_EROOR.getStatusCode();
+    super(message, TSStatusCode.NVMSPACE_MANAGER_ERROR.getStatusCode());
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java b/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java
index 175e2ab..9012d4a 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMPrimitiveMemTable.java
@@ -7,18 +7,20 @@ import java.util.Map.Entry;
 import org.apache.iotdb.db.engine.memtable.AbstractMemTable;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.engine.memtable.IWritableMemChunk;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
-import org.apache.iotdb.db.engine.memtable.WritableMemChunk;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
-import org.apache.iotdb.db.utils.datastructure.NVMTVList;
 import org.apache.iotdb.db.rescon.TVListAllocator;
-import org.apache.iotdb.db.utils.datastructure.TVList;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.db.utils.datastructure.NVMTVList;
 import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class NVMPrimitiveMemTable extends AbstractMemTable {
 
+  private static final Logger logger = LoggerFactory.getLogger(NVMPrimitiveMemTable.class);
+
   public NVMPrimitiveMemTable(String sgId) {
     super(sgId);
   }
@@ -28,9 +30,9 @@ public class NVMPrimitiveMemTable extends AbstractMemTable {
   }
 
   @Override
-  protected IWritableMemChunk genMemSeries(String deviceId, String measurementId, TSDataType dataType) {
-    return new NVMWritableMemChunk(dataType,
-        (NVMTVList) TVListAllocator.getInstance().allocate(storageGroupId, deviceId, measurementId, dataType, true));
+  protected IWritableMemChunk genMemSeries(String deviceId, String measurementId, MeasurementSchema schema) {
+    return new NVMWritableMemChunk(schema,
+        (NVMTVList) TVListAllocator.getInstance().allocate(storageGroupId, deviceId, measurementId, schema.getType(), true));
   }
 
   @Override
@@ -53,23 +55,6 @@ public class NVMPrimitiveMemTable extends AbstractMemTable {
     return this == obj;
   }
 
-  @Override
-  public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType dataType,
-      Map<String, String> props, long timeLowerBound) {
-    TimeValuePairSorter sorter;
-    if (!checkPath(deviceId, measurement)) {
-      return null;
-    } else {
-      long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
-      IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
-      IWritableMemChunk chunkCopy = new WritableMemChunk(dataType,
-          (TVList) memChunk.getTVList().clone());
-      chunkCopy.setTimeOffset(undeletedTime);
-      sorter = chunkCopy;
-    }
-    return new ReadOnlyMemChunk(dataType, sorter, props);
-  }
-
   public void loadData(Map<String, Map<String, Pair<List<NVMDataSpace>, List<NVMDataSpace>>>> dataMap) {
     if (dataMap == null) {
       return;
@@ -83,10 +68,16 @@ public class NVMPrimitiveMemTable extends AbstractMemTable {
           .entrySet()) {
         String measurementId = measurementDataEntry.getKey();
         Pair<List<NVMDataSpace>, List<NVMDataSpace>> tvListPair = measurementDataEntry.getValue();
-        TSDataType dataType = tvListPair.right.get(0).getDataType();
 
-        NVMWritableMemChunk memChunk = (NVMWritableMemChunk) createIfNotExistAndGet(deviceId, measurementId, dataType);
-        memChunk.loadData(tvListPair.left, tvListPair.right);
+        try {
+          MeasurementSchema[] schemas = MManager.getInstance().getSchemas(deviceId, new String[]{measurementId});
+          NVMWritableMemChunk memChunk = (NVMWritableMemChunk) createIfNotExistAndGet(deviceId, measurementId, schemas[0]);
+          memChunk.loadData(tvListPair.left, tvListPair.right);
+        } catch (MetadataException e) {
+          logger.error(
+              "occurs exception when reloading records from path ({}.{}): {}.(Will ignore the records)",
+              deviceId, measurementId, e.getMessage());
+        }
       }
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java
index 6a4f72e..602a418 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/memtable/NVMWritableMemChunk.java
@@ -1,276 +1,19 @@
 package org.apache.iotdb.db.nvm.memtable;
 
-import java.util.ArrayList;
 import java.util.List;
-import org.apache.iotdb.db.engine.memtable.IWritableMemChunk;
+import org.apache.iotdb.db.engine.memtable.AbstractWritableMemChunk;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.utils.datastructure.NVMTVList;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TsPrimitiveType.TsBinary;
-import org.apache.iotdb.db.utils.TsPrimitiveType.TsBoolean;
-import org.apache.iotdb.db.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.db.utils.TsPrimitiveType.TsFloat;
-import org.apache.iotdb.db.utils.TsPrimitiveType.TsInt;
-import org.apache.iotdb.db.utils.TsPrimitiveType.TsLong;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
-public class NVMWritableMemChunk implements IWritableMemChunk {
+public class NVMWritableMemChunk extends AbstractWritableMemChunk {
 
-  private static final Logger logger = LoggerFactory.getLogger(NVMWritableMemChunk.class);
-
-  private TSDataType dataType;
-  private NVMTVList list;
-  private List<TimeValuePair> sortedList;
-
-  public NVMWritableMemChunk(TSDataType dataType, NVMTVList list) {
-    this.dataType = dataType;
+  public NVMWritableMemChunk(MeasurementSchema schema, NVMTVList list) {
+    this.schema = schema;
     this.list = list;
   }
 
-  @Override
-  public void write(long insertTime, Object objectValue) {
-    switch (dataType) {
-      case BOOLEAN:
-        putBoolean(insertTime, (boolean) objectValue);
-        break;
-      case INT32:
-        putInt(insertTime, (int) objectValue);
-        break;
-      case INT64:
-        putLong(insertTime, (long) objectValue);
-        break;
-      case FLOAT:
-        putFloat(insertTime, (float) objectValue);
-        break;
-      case DOUBLE:
-        putDouble(insertTime, (double) objectValue);
-        break;
-      case TEXT:
-        putBinary(insertTime, (Binary) objectValue);
-        break;
-      default:
-        throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
-    }
-    sortedList = null;
-  }
-
-  @Override
-  public void write(long[] times, Object valueList, TSDataType dataType, List<Integer> indexes) {
-    switch (dataType) {
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) valueList;
-        if (times.length == indexes.size()) {
-          putBooleans(times, boolValues);
-          break;
-        }
-        for (Integer index : indexes) {
-          putBoolean(times[index], boolValues[index]);
-        }
-        break;
-      case INT32:
-        int[] intValues = (int[]) valueList;
-        if (times.length == indexes.size()) {
-          putInts(times, intValues);
-          break;
-        }
-        for (Integer index : indexes) {
-          putInt(times[index], intValues[index]);
-        }
-        break;
-      case INT64:
-        long[] longValues = (long[]) valueList;
-        if (times.length == indexes.size()) {
-          putLongs(times, longValues);
-          break;
-        }
-        for (Integer index : indexes) {
-          putLong(times[index], longValues[index]);
-        }
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) valueList;
-        if (times.length == indexes.size()) {
-          putFloats(times, floatValues);
-          break;
-        }
-        for (Integer index : indexes) {
-          putFloat(times[index], floatValues[index]);
-        }
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) valueList;
-        if (times.length == indexes.size()) {
-          putDoubles(times, doubleValues);
-          break;
-        }
-        for (Integer index : indexes) {
-          putDouble(times[index], doubleValues[index]);
-        }
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) valueList;
-        if (times.length == indexes.size()) {
-          putBinaries(times, binaryValues);
-          break;
-        }
-        for (Integer index : indexes) {
-          putBinary(times[index], binaryValues[index]);
-        }
-        break;
-      default:
-        throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
-    }
-    sortedList = null;
-  }
-
-
-  @Override
-  public void putLong(long t, long v) {
-    list.putLong(t, v);
-  }
-
-  @Override
-  public void putInt(long t, int v) {
-    list.putInt(t, v);
-  }
-
-  @Override
-  public void putFloat(long t, float v) {
-    list.putFloat(t, v);
-  }
-
-  @Override
-  public void putDouble(long t, double v) {
-    list.putDouble(t, v);
-  }
-
-  @Override
-  public void putBinary(long t, Binary v) {
-    list.putBinary(t, v);
-  }
-
-  @Override
-  public void putBoolean(long t, boolean v) {
-    list.putBoolean(t, v);
-  }
-
-  @Override
-  public void putLongs(long[] t, long[] v) {
-    list.putLongs(t, v);
-  }
-
-  @Override
-  public void putInts(long[] t, int[] v) {
-    list.putInts(t, v);
-  }
-
-  @Override
-  public void putFloats(long[] t, float[] v) {
-    list.putFloats(t, v);
-  }
-
-  @Override
-  public void putDoubles(long[] t, double[] v) {
-    list.putDoubles(t, v);
-  }
-
-  @Override
-  public void putBinaries(long[] t, Binary[] v) {
-    list.putBinaries(t, v);
-  }
-
-  @Override
-  public void putBooleans(long[] t, boolean[] v) {
-    list.putBooleans(t, v);
-  }
-
-  @Override
-  public synchronized NVMTVList getSortedTVList() {
-    list.sort();
-    return list;
-  }
-
-  @Override
-  public NVMTVList getTVList() {
-    return list;
-  }
-
-  @Override
-  public long count() {
-    return list.size();
-  }
-
-  @Override
-  public TSDataType getType() {
-    return dataType;
-  }
-
-  @Override
-  public void setTimeOffset(long offset) {
-    list.setTimeOffset(offset);
-  }
-
-  @Override
-  public synchronized List<TimeValuePair> getSortedTimeValuePairList() {
-    if (sortedList != null) {
-      return sortedList;
-    }
-    sortedList = new ArrayList<>();
-    list.sort();
-    for (int i = 0; i < list.size(); i++) {
-      long time = list.getTime(i);
-      if (time < list.getTimeOffset() ||
-          (i + 1 < list.size() && (time == list.getTime(i + 1)))) {
-        continue;
-      }
-      switch (dataType) {
-        case BOOLEAN:
-          sortedList.add(new TimeValuePair(time, new TsBoolean(list.getBoolean(i))));
-          break;
-        case INT32:
-          sortedList.add(new TimeValuePair(time, new TsInt(list.getInt(i))));
-          break;
-        case INT64:
-          sortedList.add(new TimeValuePair(time, new TsLong(list.getLong(i))));
-          break;
-        case FLOAT:
-          sortedList.add(new TimeValuePair(time, new TsFloat(list.getFloat(i))));
-          break;
-        case DOUBLE:
-          sortedList.add(new TimeValuePair(time, new TsDouble(list.getDouble(i))));
-          break;
-        case TEXT:
-          sortedList.add(new TimeValuePair(time, new TsBinary(list.getBinary(i))));
-          break;
-        default:
-          logger.error("Unsupported data type: {}", dataType);
-          break;
-      }
-    }
-    return this.sortedList;
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return list.size() == 0;
-  }
-
-  @Override
-  public long getMinTime() {
-    return list.getMinTime();
-  }
-
-  @Override
-  public void delete(long upperBound) {
-    list.delete(upperBound);
-  }
-
   public void loadData(List<NVMDataSpace> timeSpaceList, List<NVMDataSpace> valueSpaceList) {
-    // TODO how about abstract
-    list.loadData(timeSpaceList, valueSpaceList);
+    ((NVMTVList) list).loadData(timeSpaceList, valueSpaceList);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java b/server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java
index 1ea4f6e..81080c2 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/rescon/NVMPrimitiveArrayPool.java
@@ -4,8 +4,11 @@ import java.util.ArrayDeque;
 import java.util.EnumMap;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.nvm.exception.NVMSpaceManagerException;
+import org.apache.iotdb.db.nvm.space.NVMBinaryDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMSpaceManager;
+import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
+import org.apache.iotdb.db.utils.datastructure.NVMBinaryTVList;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 public class NVMPrimitiveArrayPool {
@@ -15,7 +18,8 @@ public class NVMPrimitiveArrayPool {
    */
   private static final EnumMap<TSDataType, ArrayDeque<NVMDataSpace>> primitiveArraysMap = new EnumMap<>(TSDataType.class);
 
-  public static final int ARRAY_SIZE = 128;
+  public static final int ARRAY_SIZE =
+      IoTDBDescriptor.getInstance().getConfig().getPrimitiveArraySize();
 
   static {
     primitiveArraysMap.put(TSDataType.BOOLEAN, new ArrayDeque());
@@ -35,17 +39,19 @@ public class NVMPrimitiveArrayPool {
   private NVMPrimitiveArrayPool() {}
 
   public synchronized NVMDataSpace getPrimitiveDataListByType(TSDataType dataType, boolean isTime) {
-    ArrayDeque<NVMDataSpace> dataListQueue = primitiveArraysMap.computeIfAbsent(dataType, k ->new ArrayDeque<>());
+    ArrayDeque<NVMDataSpace> dataListQueue = primitiveArraysMap
+        .computeIfAbsent(dataType, k -> new ArrayDeque<>());
     NVMDataSpace nvmSpace = dataListQueue.poll();
 
     if (nvmSpace == null) {
       try {
         long size = NVMSpaceManager.getPrimitiveTypeByteSize(dataType);
-        nvmSpace = NVMSpaceManager.getInstance().allocateDataSpace(size * ARRAY_SIZE, dataType, isTime);
+        nvmSpace = NVMSpaceManager.getInstance().allocateDataSpace(
+            size * (dataType == TSDataType.TEXT ? NVMBinaryDataSpace.NUM_OF_TEXT_IN_SPACE
+                : ARRAY_SIZE), dataType, isTime);
       } catch (NVMSpaceManagerException e) {
         e.printStackTrace();
         System.exit(0);
-        // TODO
       }
     }
     return nvmSpace;
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java
index abde477..3e3932f 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMBinaryDataSpace.java
@@ -8,33 +8,41 @@ import org.apache.iotdb.tsfile.utils.Binary;
 
 public class NVMBinaryDataSpace extends NVMDataSpace {
 
+  public static final int NUM_OF_TEXT_IN_SPACE = 1;
+
+  private int endPos;
+
   private int cacheSize;
   private Binary[] cachedBinaries;
+  private int[] cachedOffset;
 
   NVMBinaryDataSpace(long offset, long size, ByteBuffer byteBuffer, int index, boolean recover) {
     super(offset, size, byteBuffer, index, TSDataType.TEXT, false);
 
-    cacheSize = 0;
-    cachedBinaries = new Binary[ARRAY_SIZE];
+    reset();
+    cachedBinaries = new Binary[NUM_OF_TEXT_IN_SPACE];
+    cachedOffset = new int[NUM_OF_TEXT_IN_SPACE];
     if (recover) {
       recoverCache();
     }
   }
 
   private void recoverCache() {
-    int size = byteBuffer.getInt();
-    cacheSize = size;
-    for (int i = 0; i < size; i++) {
+    cacheSize = byteBuffer.getInt();
+    for (int i = 0; i < cacheSize; i++) {
       int len = byteBuffer.getInt();
       byte[] bytes = new byte[len];
       byteBuffer.get(bytes);
+
       cachedBinaries[i] = new Binary(bytes);
+      cachedOffset[i] = endPos;
+      endPos += len + NVMSpaceManager.getPrimitiveTypeByteSize(TSDataType.INT32);
     }
   }
 
   @Override
   public int getUnitNum() {
-    return cachedBinaries.length;
+    return cacheSize;
   }
 
   @Override
@@ -44,13 +52,21 @@ public class NVMBinaryDataSpace extends NVMDataSpace {
 
   @Override
   public void setData(int index, Object object) {
-    // todo nos support index
     Binary binary = (Binary) object;
+    cachedOffset[index] = endPos;
     cachedBinaries[index] = binary;
-    if (index >= cacheSize) {
-      byteBuffer.putInt(0, index);
-      cacheSize = index;
-    }
+    endPos = binary.getLength() + 2 * NVMSpaceManager.getPrimitiveTypeByteSize(TSDataType.INT32);
+    byteBuffer.position(NVMSpaceManager.getPrimitiveTypeByteSize(TSDataType.INT32));
+    byteBuffer.putInt(binary.getLength());
+    byteBuffer.put(binary.getValues());
+  }
+
+  public void appendData(Binary binary) {
+    cachedOffset[cacheSize] = endPos;
+    cachedBinaries[cacheSize] = binary;
+    cacheSize++;
+    endPos += binary.getLength() + NVMSpaceManager.getPrimitiveTypeByteSize(TSDataType.INT32);
+    byteBuffer.putInt(0, cacheSize);
     byteBuffer.putInt(binary.getLength());
     byteBuffer.put(binary.getValues());
   }
@@ -59,4 +75,9 @@ public class NVMBinaryDataSpace extends NVMDataSpace {
   public Object toArray() {
     return cachedBinaries;
   }
+
+  public void reset() {
+    cacheSize = 0;
+    endPos = NVMSpaceManager.getPrimitiveTypeByteSize(TSDataType.INT32);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java
index 06be125..d299b32 100644
--- a/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java
+++ b/server/src/main/java/org/apache/iotdb/db/nvm/space/NVMDataSpace.java
@@ -35,7 +35,7 @@ public class NVMDataSpace extends NVMSpace {
   }
 
   public int getValidUnitNum() {
-    // TODO only for time
+    // only for time space
     int count = 0;
     while (count < unitNum) {
       long v = (long) getData(count);
@@ -67,7 +67,7 @@ public class NVMDataSpace extends NVMSpace {
         object = byteBuffer.getDouble(index);
         break;
       case TEXT:
-        // TODO
+        // override in NVMBinaryDataSpace
         break;
     }
     return object;
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 04e80f2..fae41c4 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
@@ -231,6 +231,7 @@ public class AggregationExecutor {
         continue;
       }
       BatchData nextOverlappedPageData = seriesReader.nextPage();
+      System.out.println("read:" + nextOverlappedPageData.count);
       for (int i = 0; i < aggregateResultList.size(); i++) {
         if (!isCalculatedArray[i]) {
           AggregateResult aggregateResult = aggregateResultList.get(i);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
index 816be29..b1e2eb6 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
@@ -477,6 +477,7 @@ class SeriesReader {
            * get current first point in mergeReader, this maybe overlapped latter
            */
           TimeValuePair timeValuePair = mergeReader.currentTimeValuePair();
+          System.out.println("read:" + timeValuePair);
 
           if (timeValuePair.getTimestamp() > currentPageEndTime) {
             break;
diff --git a/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java b/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
index fdb6728..23f2a90 100644
--- a/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
+++ b/server/src/main/java/org/apache/iotdb/db/rescon/MemTablePool.java
@@ -42,7 +42,6 @@ public class MemTablePool {
   private int nvmSize = 0;
 
   private static final int WAIT_TIME = 2000;
-  private int size = 0;
 
   private MemTablePool() {
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
index 507be77..d8728f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AbstractTVList.java
@@ -80,6 +80,30 @@ public abstract class AbstractTVList {
     throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
   }
 
+  public void putLongs(long[] time, long[] value, int start, int end) {
+    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+  }
+
+  public void putInts(long[] time, int[] value, int start, int end) {
+    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+  }
+
+  public void putFloats(long[] time, float[] value, int start, int end) {
+    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+  }
+
+  public void putDoubles(long[] time, double[] value, int start, int end) {
+    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+  }
+
+  public void putBinaries(long[] time, Binary[] value, int start, int end) {
+    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+  }
+
+  public void putBooleans(long[] time, boolean[] value, int start, int end) {
+    throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
+  }
+
   public long getLong(int index) {
     throw new UnsupportedOperationException(ERR_DATATYPE_NOT_CONSISTENT);
   }
@@ -260,6 +284,7 @@ public abstract class AbstractTVList {
       int n = start - left;  // The number of elements to move
       for (int i = n; i >= 1; i--) {
         set(left + i - 1, left + i);
+        setToSorted(left + i - 1, left + i);
       }
       setPivotTo(left);
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
index f146d28..3b6b89f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java
@@ -262,11 +262,6 @@ public class BinaryTVList extends TVList {
   }
 
   @Override
-  protected Object getValueForSort(int index) {
-    return getBinary(index);
-  }
-
-  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     set(index, timestamp, (Binary) value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
index ce42960..0ad2ecc 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java
@@ -262,11 +262,6 @@ public class BooleanTVList extends TVList {
   }
 
   @Override
-  protected Object getValueForSort(int index) {
-    return getBoolean(index);
-  }
-
-  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     set(index, timestamp, (boolean) value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java
index 131e9fb..90a39fc 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java
@@ -266,11 +266,6 @@ public class DoubleTVList extends TVList {
   }
 
   @Override
-  protected Object getValueForSort(int index) {
-    return getDouble(index);
-  }
-
-  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     set(index, timestamp, (double) value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
index 6861365..9ea5086 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java
@@ -269,11 +269,6 @@ public class FloatTVList extends TVList {
   }
 
   @Override
-  protected Object getValueForSort(int index) {
-    return getFloat(index);
-  }
-
-  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     set(index, timestamp, (float) value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
index f3c73bb..75d2248 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java
@@ -262,11 +262,6 @@ public class IntTVList extends TVList {
   }
 
   @Override
-  protected Object getValueForSort(int index) {
-    return getInt(index);
-  }
-
-  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     set(index, timestamp, (int) value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
index 9f8a0ee..5922caa 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java
@@ -261,11 +261,6 @@ public class LongTVList extends TVList {
   }
 
   @Override
-  protected Object getValueForSort(int index) {
-    return getLong(index);
-  }
-
-  @Override
   protected void setForSort(int index, long timestamp, Object value) {
     set(index, timestamp, (long) value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
index 0b8ce51..88c37fc 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBinaryTVList.java
@@ -4,6 +4,7 @@ import static org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool.ARRAY_SIZE;
 
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.nvm.rescon.NVMPrimitiveArrayPool;
+import org.apache.iotdb.db.nvm.space.NVMBinaryDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMSpaceManager;
 import org.apache.iotdb.db.nvm.space.NVMSpaceMetadataManager;
@@ -14,7 +15,6 @@ import org.apache.iotdb.tsfile.utils.Binary;
 // TODO how to organize data
 public class NVMBinaryTVList extends NVMTVList {
 
-  // TODO
   private Binary[][] sortedValues;
   private Binary[][] tempValuesForSort;
 
@@ -27,10 +27,9 @@ public class NVMBinaryTVList extends NVMTVList {
   public void putBinary(long timestamp, Binary value) {
     checkExpansion();
     int arrayIndex = size;
-    int elementIndex = 0;
     minTime = minTime <= timestamp ? minTime : timestamp;
-    timestamps.get(arrayIndex).setData(elementIndex, timestamp);
-    values.get(arrayIndex).setData(elementIndex, value);
+    timestamps.get(arrayIndex).setData(0, timestamp);
+    ((NVMBinaryDataSpace)values.get(arrayIndex)).appendData(value);
     size++;
     if (sorted && size > 1 && timestamp < getTime(size - 2)) {
       sorted = false;
@@ -39,7 +38,8 @@ public class NVMBinaryTVList extends NVMTVList {
 
   @Override
   protected void checkExpansion() {
-    NVMDataSpace valueSpace = expandValues();
+    NVMBinaryDataSpace valueSpace = (NVMBinaryDataSpace) expandValues();
+    valueSpace.reset();
     NVMDataSpace timeSpace = NVMPrimitiveArrayPool
         .getInstance().getPrimitiveDataListByType(TSDataType.INT64, true);
     timestamps.add(timeSpace);
@@ -48,22 +48,46 @@ public class NVMBinaryTVList extends NVMTVList {
   }
 
   @Override
+  public long getTime(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    return (long) timestamps.get(index).getData(0);
+  }
+
+  @Override
   public Binary getBinary(int index) {
+    return (Binary) getValue(index);
+  }
+
+  @Override
+  public Object getValue(int index) {
     if (index >= size) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
-    int arrayIndex = index;
-    int elementIndex = 0;
-    return (Binary) values.get(arrayIndex).getData(elementIndex);
+    return values.get(index).getData(0);
+  }
+
+  @Override
+  protected void set(int index, long timestamp, Object value) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    timestamps.get(index).setData(0, timestamp);
+    values.get(index).setData(0, value);
   }
 
   @Override
   public BinaryTVList clone() {
     BinaryTVList cloneList = new BinaryTVList();
-    cloneAs(cloneList);
-    for (NVMDataSpace valueSpace : values) {
-      cloneList.addBatchValue((Binary[]) cloneValue(valueSpace));
+
+    long[] cloneTimestamps = new long[size];
+    Binary[] cloneValues = new Binary[size];
+    for (int i = 0; i < size; i++) {
+      cloneTimestamps[i] = timestamps.get(i).getLong(0);
+      cloneValues[i] = (Binary) values.get(i).getData(0);
     }
+    cloneList.putBinaries(cloneTimestamps, cloneValues);
     return cloneList;
   }
 
@@ -172,49 +196,17 @@ public class NVMBinaryTVList extends NVMTVList {
   
   @Override
   public void putBinaries(long[] time, Binary[] value) {
-    checkExpansion();
-    int idx = 0;
     int length = time.length;
 
     for (int i = 0; i < length; i++) {
       putBinary(time[i], value[i]);
     }
+  }
 
-//    updateMinTimeAndSorted(time);
-//
-//    while (idx < length) {
-//      int inputRemaining = length - idx;
-//      int arrayIdx = size / ARRAY_SIZE;
-//      int elementIdx = size % ARRAY_SIZE;
-//      int internalRemaining  = ARRAY_SIZE - elementIdx;
-//      if (internalRemaining >= inputRemaining) {
-//        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-//        size += inputRemaining;
-//        break;
-//      } else {
-//        // the remaining inputs cannot fit the last array, fill the last array and create a new
-//        // one and enter the next loop
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-//        idx += internalRemaining;
-//        size += internalRemaining;
-//        checkExpansion();
-//      }
-//    }
-  }
-
-  public static void main(String[] args) throws StartupException {
-    NVMSpaceManager.getInstance().init();
-
-    NVMBinaryTVList tvList = new NVMBinaryTVList("sg", "d0", "s0");
-    int size = 5000;
-    for (int i = 0; i < size; i++) {
-      String v = String.valueOf(size - i);
-      tvList.putBinary(i, Binary.valueOf(v));
+  @Override
+  public void putBinaries(long[] time, Binary[] value, int start, int end) {
+    for (int i = start; i < end; i++) {
+      putBinary(time[i], value[i]);
     }
-
-    tvList.sort();
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
index 3c8a8e7..726fb1e 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMBooleanTVList.java
@@ -33,12 +33,7 @@ public class NVMBooleanTVList extends NVMTVList {
 
   @Override
   public boolean getBoolean(int index) {
-    if (index >= size) {
-      throw new ArrayIndexOutOfBoundsException(index);
-    }
-    int arrayIndex = index / ARRAY_SIZE;
-    int elementIndex = index % ARRAY_SIZE;
-    return (boolean) values.get(arrayIndex).getData(elementIndex);
+    return (boolean) getValue(index);
   }
 
   @Override
@@ -156,36 +151,17 @@ public class NVMBooleanTVList extends NVMTVList {
   
   @Override
   public void putBooleans(long[] time, boolean[] value) {
-    checkExpansion();
-    int idx = 0;
     int length = time.length;
 
     for (int i = 0; i < length; i++) {
       putBoolean(time[i], value[i]);
     }
+  }
 
-//    updateMinTimeAndSorted(time);
-//
-//    while (idx < length) {
-//      int inputRemaining = length - idx;
-//      int arrayIdx = size / ARRAY_SIZE;
-//      int elementIdx = size % ARRAY_SIZE;
-//      int internalRemaining  = ARRAY_SIZE - elementIdx;
-//      if (internalRemaining >= inputRemaining) {
-//        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-//        size += inputRemaining;
-//        break;
-//      } else {
-//        // the remaining inputs cannot fit the last array, fill the last array and create a new
-//        // one and enter the next loop
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-//        idx += internalRemaining;
-//        size += internalRemaining;
-//        checkExpansion();
-//      }
-//    }
+  @Override
+  public void putBooleans(long[] time, boolean[] value, int start, int end) {
+    for (int i = start; i < end; i++) {
+      putBoolean(time[i], value[i]);
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
index 7455d18..c8bdee4 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMDoubleTVList.java
@@ -34,12 +34,7 @@ public class NVMDoubleTVList extends NVMTVList {
 
   @Override
   public double getDouble(int index) {
-    if (index >= size) {
-      throw new ArrayIndexOutOfBoundsException(index);
-    }
-    int arrayIndex = index / ARRAY_SIZE;
-    int elementIndex = index % ARRAY_SIZE;
-    return (double) values.get(arrayIndex).getData(elementIndex);
+    return (double) getValue(index);
   }
 
   @Override
@@ -157,45 +152,17 @@ public class NVMDoubleTVList extends NVMTVList {
 
   @Override
   public void putDoubles(long[] time, double[] value) {
-//    checkExpansion();
-    int idx = 0;
     int length = time.length;
 
     for (int i = 0; i < length; i++) {
       putDouble(time[i], value[i]);
     }
+  }
 
-//    long startTime = System.currentTimeMillis();
-//    for (NVMDataSpace timeSpace : timestamps) {
-//      timeSpace.force();
-//    }
-//    for (NVMDataSpace valueSpace : values) {
-//      valueSpace.force();
-//    }
-//    PerfMonitor.add("NVMTVList.force", System.currentTimeMillis() - startTime);
-
-//    updateMinTimeAndSorted(time);
-//
-//    while (idx < length) {
-//      int inputRemaining = length - idx;
-//      int arrayIdx = size / ARRAY_SIZE;
-//      int elementIdx = size % ARRAY_SIZE;
-//      int internalRemaining  = ARRAY_SIZE - elementIdx;
-//      if (internalRemaining >= inputRemaining) {
-//        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-//        size += inputRemaining;
-//        break;
-//      } else {
-//        // the remaining inputs cannot fit the last array, fill the last array and create a new
-//        // one and enter the next loop
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-//        idx += internalRemaining;
-//        size += internalRemaining;
-//        checkExpansion();
-//      }
-//    }
+  @Override
+  public void putDoubles(long[] time, double[] value, int start, int end) {
+    for (int i = start; i < end; i++) {
+      putDouble(time[i], value[i]);
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
index b9814e8..d72e05a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMFloatTVList.java
@@ -33,12 +33,7 @@ public class NVMFloatTVList extends NVMTVList {
 
   @Override
   public float getFloat(int index) {
-    if (index >= size) {
-      throw new ArrayIndexOutOfBoundsException(index);
-    }
-    int arrayIndex = index / ARRAY_SIZE;
-    int elementIndex = index % ARRAY_SIZE;
-    return (float) values.get(arrayIndex).getData(elementIndex);
+    return (float) getValue(index);
   }
 
   @Override
@@ -156,36 +151,17 @@ public class NVMFloatTVList extends NVMTVList {
 
   @Override
   public void putFloats(long[] time, float[] value) {
-    checkExpansion();
-    int idx = 0;
     int length = time.length;
 
     for (int i = 0; i < length; i++) {
       putFloat(time[i], value[i]);
     }
+  }
 
-//    updateMinTimeAndSorted(time);
-//
-//    while (idx < length) {
-//      int inputRemaining = length - idx;
-//      int arrayIdx = size / ARRAY_SIZE;
-//      int elementIdx = size % ARRAY_SIZE;
-//      int internalRemaining  = ARRAY_SIZE - elementIdx;
-//      if (internalRemaining >= inputRemaining) {
-//        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-//        size += inputRemaining;
-//        break;
-//      } else {
-//        // the remaining inputs cannot fit the last array, fill the last array and create a new
-//        // one and enter the next loop
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-//        idx += internalRemaining;
-//        size += internalRemaining;
-//        checkExpansion();
-//      }
-//    }
+  @Override
+  public void putFloats(long[] time, float[] value, int start, int end) {
+    for (int i = start; i < end; i++) {
+      putFloat(time[i], value[i]);
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
index 5ff1403..13e5e0a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMIntTVList.java
@@ -33,12 +33,7 @@ public class NVMIntTVList extends NVMTVList {
 
   @Override
   public int getInt(int index) {
-    if (index >= size) {
-      throw new ArrayIndexOutOfBoundsException(index);
-    }
-    int arrayIndex = index / ARRAY_SIZE;
-    int elementIndex = index % ARRAY_SIZE;
-    return (int) values.get(arrayIndex).getData(elementIndex);
+    return (int) getValue(index);
   }
 
   @Override
@@ -156,36 +151,17 @@ public class NVMIntTVList extends NVMTVList {
 
   @Override
   public void putInts(long[] time, int[] value) {
-    checkExpansion();
-    int idx = 0;
     int length = time.length;
 
     for (int i = 0; i < length; i++) {
       putInt(time[i], value[i]);
     }
+  }
 
-//    updateMinTimeAndSorted(time);
-//
-//    while (idx < length) {
-//      int inputRemaining = length - idx;
-//      int arrayIdx = size / ARRAY_SIZE;
-//      int elementIdx = size % ARRAY_SIZE;
-//      int internalRemaining  = ARRAY_SIZE - elementIdx;
-//      if (internalRemaining >= inputRemaining) {
-//        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-//        size += inputRemaining;
-//        break;
-//      } else {
-//        // the remaining inputs cannot fit the last array, fill the last array and create a new
-//        // one and enter the next loop
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-//        idx += internalRemaining;
-//        size += internalRemaining;
-//        checkExpansion();
-//      }
-//    }
+  @Override
+  public void putInts(long[] time, int[] value, int start, int end) {
+    for (int i = start; i < end; i++) {
+      putInt(time[i], value[i]);
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
index 956880f..98823be 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMLongTVList.java
@@ -33,12 +33,7 @@ public class NVMLongTVList extends NVMTVList {
 
   @Override
   public long getLong(int index) {
-    if (index >= size) {
-      throw new ArrayIndexOutOfBoundsException(index);
-    }
-    int arrayIndex = index / ARRAY_SIZE;
-    int elementIndex = index % ARRAY_SIZE;
-    return (long) values.get(arrayIndex).getData(elementIndex);
+    return (long) getValue(index);
   }
 
   @Override
@@ -156,37 +151,18 @@ public class NVMLongTVList extends NVMTVList {
 
   @Override
   public void putLongs(long[] time, long[] value) {
-    checkExpansion();
-    int idx = 0;
     int length = time.length;
 
     for (int i = 0; i < length; i++) {
       putLong(time[i], value[i]);
     }
+  }
 
-//    updateMinTimeAndSorted(time);
-//
-//    while (idx < length) {
-//      int inputRemaining = length - idx;
-//      int arrayIdx = size / ARRAY_SIZE;
-//      int elementIdx = size % ARRAY_SIZE;
-//      int internalRemaining  = ARRAY_SIZE - elementIdx;
-//      if (internalRemaining >= inputRemaining) {
-//        // the remaining inputs can fit the last array, copy all remaining inputs into last array
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, inputRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-//        size += inputRemaining;
-//        break;
-//      } else {
-//        // the remaining inputs cannot fit the last array, fill the last array and create a new
-//        // one and enter the next loop
-//        System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
-//        System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
-//        idx += internalRemaining;
-//        size += internalRemaining;
-//        checkExpansion();
-//      }
-//    }
+  @Override
+  public void putLongs(long[] time, long[] value, int start, int end) {
+    for (int i = start; i < end; i++) {
+      putLong(time[i], value[i]);
+    }
   }
 }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
index eda0cb6..6e4d2d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/NVMTVList.java
@@ -9,6 +9,9 @@ import org.apache.iotdb.db.nvm.space.NVMDataSpace;
 import org.apache.iotdb.db.nvm.space.NVMSpaceMetadataManager;
 import org.apache.iotdb.db.rescon.PrimitiveArrayPool;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
 public abstract class NVMTVList extends AbstractTVList {
 
@@ -102,7 +105,7 @@ public abstract class NVMTVList extends AbstractTVList {
   }
 
   @Override
-  public void delete(long upperBound) {
+  public int delete(long upperBound) {
     int newSize = 0;
     minTime = Long.MAX_VALUE;
     for (int i = 0; i < size; i++) {
@@ -112,6 +115,7 @@ public abstract class NVMTVList extends AbstractTVList {
         minTime = time < minTime ? time : minTime;
       }
     }
+    int deletedNumber = size - newSize;
     size = newSize;
     // release primitive arrays that are empty
     int newArrayNum = newSize / ARRAY_SIZE;
@@ -122,6 +126,7 @@ public abstract class NVMTVList extends AbstractTVList {
       releaseLastTimeArray();
       releaseLastValueArray();
     }
+    return deletedNumber;
   }
 
   @Override
@@ -279,6 +284,17 @@ public abstract class NVMTVList extends AbstractTVList {
     return tempTimestampsForSort[arrayIndex][elementIndex];
   }
 
+  protected abstract Object getValueForSort(int index);
+
+  public Object getValue(int index) {
+    if (index >= size) {
+      throw new ArrayIndexOutOfBoundsException(index);
+    }
+    int arrayIndex = index / ARRAY_SIZE;
+    int elementIndex = index % ARRAY_SIZE;
+    return values.get(arrayIndex).getData(elementIndex);
+  }
+
   @Override
   protected void setForSort(int index, long timestamp, Object value) {
     if (index >= size) {
@@ -302,4 +318,17 @@ public abstract class NVMTVList extends AbstractTVList {
   protected void setPivotTo(int pos) {
     setForSort(pos, pivotTime, pivotValue);
   }
+
+  @Override
+  public TimeValuePair getTimeValuePair(int index) {
+    return new TimeValuePair(getTime(index),
+        TsPrimitiveType.getByType(dataType, getValue(index)));
+  }
+
+  @Override
+  protected TimeValuePair getTimeValuePair(int index, long time, Integer floatPrecision,
+      TSEncoding encoding) {
+    return new TimeValuePair(time, TsPrimitiveType.getByType(dataType, getValue(index)));
+
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
index 7663065..d9dfcd3 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
@@ -178,7 +178,6 @@ public class LogReplayer {
           MManager.getInstance().getSchemas(insertPlan.getDeviceId(), insertPlan.getMeasurements());
       insertPlan.setSchemasAndTransferType(schemas);
       recoverMemTable.insert(insertPlan);
-      // TODO how about NVM
     } catch (Exception e) {
       logger.error(
           "occurs exception when replaying the record {} at timestamp {}: {}.(Will ignore the record)",
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 05b6553..425d0d3 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
@@ -73,8 +73,8 @@ public class TsFileRecoverPerformer {
     this.versionController = versionController;
     this.resource = currentTsFileResource;
     this.acceptUnseq = acceptUnseq;
-    this.storageGroupId = sgId;
     this.isLastFile = isLastFile;
+    this.storageGroupId = sgId;
   }
 
   /**
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java
index f180ec2..26bc141 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java
@@ -85,14 +85,12 @@ public class IoTDBAggregationIT {
 
   @Before
   public void setUp() throws Exception {
-    System.out.println("set up start");
     EnvironmentUtils.closeStatMonitor();
     prevPartitionInterval = IoTDBDescriptor.getInstance().getConfig().getPartitionInterval();
     IoTDBDescriptor.getInstance().getConfig().setPartitionInterval(1000);
     EnvironmentUtils.envSetUp();
     Class.forName(Config.JDBC_DRIVER_NAME);
     prepareData();
-    System.out.println("set up end");
   }
 
   @After
@@ -105,7 +103,6 @@ public class IoTDBAggregationIT {
   //details in: https://issues.apache.org/jira/projects/IOTDB/issues/IOTDB-54
   @Test
   public void test() throws SQLException {
-    System.out.println("test start");
     String[] retArray = new String[]{
         "0,2",
         "0,4",
@@ -167,7 +164,6 @@ public class IoTDBAggregationIT {
 
   @Test
   public void countTest() throws SQLException {
-    System.out.println("countTest start");
     String[] retArray = new String[]{
         "0,2001,2001,2001,2001",
         "0,7500,7500,7500,7500"
@@ -214,7 +210,6 @@ public class IoTDBAggregationIT {
 
   @Test
   public void firstTest() {
-    System.out.println("firstTest start");
     String[] retArray = new String[]{
         "0,2000,2000,2000.0,2000",
         "0,500,500,500.0,500"
@@ -269,7 +264,6 @@ public class IoTDBAggregationIT {
 
   @Test
   public void lastTest() throws SQLException {
-    System.out.println("lastTest start");
     String[] retArray = new String[]{
         "0,8499,8499.0",
         "0,1499,1499.0",
@@ -333,7 +327,6 @@ public class IoTDBAggregationIT {
 
   @Test
   public void maxminTimeTest() throws SQLException {
-    System.out.println("maxminTimeTest start");
     String[] retArray = new String[]{
         "0,8499,500",
         "0,2499,2000"
@@ -428,7 +421,6 @@ public class IoTDBAggregationIT {
 
   @Test
   public void avgSumTest() {
-    System.out.println("avgSumTest start");
     String[] retArray = new String[]{
         "0,1.4508E7,7250.374812593702",
         "0,626750.0,1250.9980039920158"
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index 6ceb46d..955ee4d 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -37,6 +37,7 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.ChunkMetadataCache;
+import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.monitor.StatMonitor;
@@ -212,7 +213,12 @@ public class EnvironmentUtils {
     TEST_QUERY_JOB_ID = QueryResourceManager.getInstance().assignQueryId(true);
     TEST_QUERY_CONTEXT = new QueryContext(TEST_QUERY_JOB_ID);
 
-    NVMSpaceManager.getInstance().init();
+    try {
+      NVMSpaceManager.getInstance().init();
+    } catch (StartupException e) {
+      logger.error("init NVM space failed", e);
+      fail(e.getMessage());
+    }
   }
 
   public static void stopDaemon() {
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
index 00e3e4c..1a4857d 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/RecoverResourceFromReaderTest.java
@@ -182,7 +182,7 @@ public class RecoverResourceFromReaderTest {
     }
 
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix,
-        versionController, resource, true, false);
+        versionController, resource, true, false, "root.sg");
     ActiveTimeSeriesCounter.getInstance()
         .init(resource.getFile().getParentFile().getParentFile().getName());
     performer.recover();
diff --git a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
index 0e34574..c448855 100644
--- a/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/writelog/recover/SeqTsFileRecoverTest.java
@@ -219,7 +219,7 @@ public class SeqTsFileRecoverTest {
   @Test
   public void testLastRecovery() throws StorageGroupProcessorException, IOException {
     TsFileRecoverPerformer performer = new TsFileRecoverPerformer(logNodePrefix,
-        versionController, resource, true, true);
+        versionController, resource, true, true, "root.sg");
     ActiveTimeSeriesCounter.getInstance().init(storageGroup);
     RestorableTsFileIOWriter writer = performer.recover();
 
diff --git a/server/src/test/resources/iotdb-engine.properties b/server/src/test/resources/iotdb-engine.properties
index b213e4c..c792aba 100644
--- a/server/src/test/resources/iotdb-engine.properties
+++ b/server/src/test/resources/iotdb-engine.properties
@@ -18,4 +18,5 @@
 
 base_dir=target/tmp
 data_dirs=target/data
-wal_dir=target/wal
\ No newline at end of file
+wal_dir=target/wal
+nvm_dir=target/nvm
\ No newline at end of file
diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
index af191b9..5d4e96e 100644
--- a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
+++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
@@ -43,7 +43,7 @@ public enum TSStatusCode {
   TSFILE_PROCESSOR_ERROR(314),
   PATH_ILLEGAL(315),
   LOAD_FILE_ERROR(316),
-  NVMSPACE_MANAGER_EROOR(317),
+  NVMSPACE_MANAGER_ERROR(317),
 
   EXECUTE_STATEMENT_ERROR(400),
   SQL_PARSE_ERROR(401),
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 a893c1b..da9dd83 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
@@ -68,7 +68,7 @@ public class BatchData implements Serializable {
   private int writeCurArrayIndex;
 
   // the insert timestamp number of timeRet
-  private int count;
+  public int count;
 
 
   private List<long[]> timeRet;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
index aa334d8..4289952 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java
@@ -93,6 +93,7 @@ public class ChunkReader implements IChunkReader {
       // deserialize a PageHeader from chunkDataBuffer
       PageHeader pageHeader = isFromOldTsFile ? HeaderUtils.deserializePageHeaderV1(chunkDataBuffer, chunkHeader.getDataType()) :
           PageHeader.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType());
+      System.out.println("statistics:" + pageHeader.getStatistics());
       // if the current page satisfies
       if (pageSatisfied(pageHeader)) {
         pageReaderList.add(constructPageReaderForNextPage(pageHeader));