You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/01/20 05:02:58 UTC

[iotdb] branch master updated: [IOTDB-2433] Fix aligned timeseries mem control bug 2 (#4915)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5d09b8c  [IOTDB-2433] Fix aligned timeseries mem control bug 2 (#4915)
5d09b8c is described below

commit 5d09b8cc46be44e9b7771f8a02543d0b79848768
Author: Haonan <hh...@outlook.com>
AuthorDate: Thu Jan 20 13:02:22 2022 +0800

    [IOTDB-2433] Fix aligned timeseries mem control bug 2 (#4915)
---
 .../iotdb/db/engine/memtable/AbstractMemTable.java |  4 +-
 .../engine/memtable/AlignedWritableMemChunk.java   | 14 ++--
 .../memtable/AlignedWritableMemChunkGroup.java     |  4 +-
 .../apache/iotdb/db/engine/memtable/IMemTable.java |  2 +-
 .../db/engine/memtable/IWritableMemChunkGroup.java |  2 +-
 .../iotdb/db/engine/memtable/WritableMemChunk.java | 16 ++--
 .../db/engine/memtable/WritableMemChunkGroup.java  |  4 +-
 .../querycontext/AlignedReadOnlyMemChunk.java      |  4 +-
 .../db/engine/storagegroup/TsFileProcessor.java    |  8 +-
 .../apache/iotdb/db/metadata/path/AlignedPath.java |  2 +-
 .../iotdb/db/metadata/path/MeasurementPath.java    |  2 +-
 .../db/utils/datastructure/AlignedTVList.java      | 51 ++++++------
 .../iotdb/db/utils/datastructure/BinaryTVList.java | 32 +++----
 .../db/utils/datastructure/BooleanTVList.java      | 32 +++----
 .../iotdb/db/utils/datastructure/DoubleTVList.java | 32 +++----
 .../iotdb/db/utils/datastructure/FloatTVList.java  | 32 +++----
 .../iotdb/db/utils/datastructure/IntTVList.java    | 33 ++++----
 .../iotdb/db/utils/datastructure/LongTVList.java   | 33 ++++----
 .../iotdb/db/utils/datastructure/TVList.java       | 30 +++----
 .../engine/storagegroup/TsFileProcessorTest.java   | 97 ++++++++++++++--------
 .../db/utils/datastructure/BinaryTVListTest.java   |  8 +-
 .../db/utils/datastructure/BooleanTVListTest.java  | 10 +--
 .../db/utils/datastructure/DoubleTVListTest.java   | 12 +--
 .../db/utils/datastructure/FloatTVListTest.java    | 12 +--
 .../db/utils/datastructure/IntTVListTest.java      | 12 +--
 .../db/utils/datastructure/LongTVListTest.java     | 14 ++--
 .../db/utils/datastructure/VectorTVListTest.java   | 14 ++--
 27 files changed, 272 insertions(+), 244 deletions(-)

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 e491a87..57116e9 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
@@ -349,9 +349,9 @@ public abstract class AbstractMemTable implements IMemTable {
   }
 
   @Override
-  public long getCurrentChunkPointNum(IDeviceID deviceId, String measurement) {
+  public long getCurrentTVListSize(IDeviceID deviceId, String measurement) {
     IWritableMemChunkGroup memChunkGroup = memTableMap.get(deviceId);
-    return memChunkGroup.getCurrentChunkPointNum(measurement);
+    return memChunkGroup.getCurrentTVListSize(measurement);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java
index 99edd4a..2339f51 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunk.java
@@ -192,11 +192,11 @@ public class AlignedWritableMemChunk implements IWritableMemChunk {
 
   @Override
   public long count() {
-    return (long) list.size() * measurementIndexMap.size();
+    return (long) list.rowCount() * measurementIndexMap.size();
   }
 
   public long alignedListSize() {
-    return list.size();
+    return list.rowCount();
   }
 
   @Override
@@ -270,11 +270,11 @@ public class AlignedWritableMemChunk implements IWritableMemChunk {
   public void encode(IChunkWriter chunkWriter) {
     AlignedChunkWriterImpl alignedChunkWriter = (AlignedChunkWriterImpl) chunkWriter;
     List<Integer> timeDuplicateAlignedRowIndexList = null;
-    for (int sortedRowIndex = 0; sortedRowIndex < list.size(); sortedRowIndex++) {
+    for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) {
       long time = list.getTime(sortedRowIndex);
 
       // skip duplicated data
-      if ((sortedRowIndex + 1 < list.size() && (time == list.getTime(sortedRowIndex + 1)))) {
+      if ((sortedRowIndex + 1 < list.rowCount() && (time == list.getTime(sortedRowIndex + 1)))) {
         // record the time duplicated row index list for vector type
         if (timeDuplicateAlignedRowIndexList == null) {
           timeDuplicateAlignedRowIndexList = new ArrayList<>();
@@ -340,7 +340,7 @@ public class AlignedWritableMemChunk implements IWritableMemChunk {
 
   @Override
   public long getFirstPoint() {
-    if (list.size() == 0) {
+    if (list.rowCount() == 0) {
       return Long.MAX_VALUE;
     }
     return getSortedTvListForQuery().getTimeValuePair(0).getTimestamp();
@@ -348,11 +348,11 @@ public class AlignedWritableMemChunk implements IWritableMemChunk {
 
   @Override
   public long getLastPoint() {
-    if (list.size() == 0) {
+    if (list.rowCount() == 0) {
       return Long.MIN_VALUE;
     }
     return getSortedTvListForQuery()
-        .getTimeValuePair(getSortedTvListForQuery().size() - 1)
+        .getTimeValuePair(getSortedTvListForQuery().rowCount() - 1)
         .getTimestamp();
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java
index 795f9c0..56fc7e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AlignedWritableMemChunkGroup.java
@@ -110,8 +110,8 @@ public class AlignedWritableMemChunkGroup implements IWritableMemChunkGroup {
   }
 
   @Override
-  public long getCurrentChunkPointNum(String measurement) {
-    return memChunk.count();
+  public long getCurrentTVListSize(String measurement) {
+    return memChunk.getTVList().rowCount();
   }
 
   public AlignedWritableMemChunk getAlignedMemChunk() {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
index 3e2a515..69aeb47 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
@@ -155,7 +155,7 @@ public interface IMemTable {
   boolean checkIfChunkDoesNotExist(IDeviceID deviceId, String measurement);
 
   /** only used when mem control enabled */
-  long getCurrentChunkPointNum(IDeviceID deviceId, String measurement);
+  long getCurrentTVListSize(IDeviceID deviceId, String measurement);
 
   /** only used when mem control enabled */
   void addTextDataSize(long textDataIncrement);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java
index 00bcf7c..0ee7075 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunkGroup.java
@@ -49,5 +49,5 @@ public interface IWritableMemChunkGroup {
   int delete(
       PartialPath originalPath, PartialPath devicePath, long startTimestamp, long endTimestamp);
 
-  long getCurrentChunkPointNum(String measurement);
+  long getCurrentTVListSize(String measurement);
 }
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 a6c4355..5f99bb6 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
@@ -227,7 +227,7 @@ public class WritableMemChunk implements IWritableMemChunk {
 
   @Override
   public long count() {
-    return list.size();
+    return list.rowCount();
   }
 
   @Override
@@ -242,7 +242,7 @@ public class WritableMemChunk implements IWritableMemChunk {
 
   @Override
   public long getFirstPoint() {
-    if (list.size() == 0) {
+    if (list.rowCount() == 0) {
       return Long.MAX_VALUE;
     }
     return getSortedTvListForQuery().getTimeValuePair(0).getTimestamp();
@@ -250,11 +250,11 @@ public class WritableMemChunk implements IWritableMemChunk {
 
   @Override
   public long getLastPoint() {
-    if (list.size() == 0) {
+    if (list.rowCount() == 0) {
       return Long.MIN_VALUE;
     }
     return getSortedTvListForQuery()
-        .getTimeValuePair(getSortedTvListForQuery().size() - 1)
+        .getTimeValuePair(getSortedTvListForQuery().rowCount() - 1)
         .getTimestamp();
   }
 
@@ -270,7 +270,7 @@ public class WritableMemChunk implements IWritableMemChunk {
 
   @Override
   public String toString() {
-    int size = list.size();
+    int size = list.rowCount();
     int firstIndex = 0;
     int lastIndex = size - 1;
     long minTime = Long.MAX_VALUE;
@@ -305,16 +305,16 @@ public class WritableMemChunk implements IWritableMemChunk {
 
     ChunkWriterImpl chunkWriterImpl = (ChunkWriterImpl) chunkWriter;
 
-    for (int sortedRowIndex = 0; sortedRowIndex < list.size(); sortedRowIndex++) {
+    for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) {
       long time = list.getTime(sortedRowIndex);
 
       // skip duplicated data
-      if ((sortedRowIndex + 1 < list.size() && (time == list.getTime(sortedRowIndex + 1)))) {
+      if ((sortedRowIndex + 1 < list.rowCount() && (time == list.getTime(sortedRowIndex + 1)))) {
         continue;
       }
 
       // store last point for SDT
-      if (sortedRowIndex + 1 == list.size()) {
+      if (sortedRowIndex + 1 == list.rowCount()) {
         ((ChunkWriterImpl) chunkWriterImpl).setLastPoint(true);
       }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java
index 578a346..820fa41 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunkGroup.java
@@ -131,7 +131,7 @@ public class WritableMemChunkGroup implements IWritableMemChunkGroup {
   }
 
   @Override
-  public long getCurrentChunkPointNum(String measurement) {
-    return memChunkMap.get(measurement).count();
+  public long getCurrentTVListSize(String measurement) {
+    return memChunkMap.get(measurement).getTVList().rowCount();
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/AlignedReadOnlyMemChunk.java b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/AlignedReadOnlyMemChunk.java
index d050faa..f0d7cb1 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/querycontext/AlignedReadOnlyMemChunk.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/querycontext/AlignedReadOnlyMemChunk.java
@@ -95,7 +95,7 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk {
         new ChunkMetadata(measurementUid, TSDataType.VECTOR, 0, timeStatistics);
     List<IChunkMetadata> valueChunkMetadataList = new ArrayList<>();
     // update time chunk
-    for (int row = 0; row < alignedChunkData.size(); row++) {
+    for (int row = 0; row < alignedChunkData.rowCount(); row++) {
       timeStatistics.update(alignedChunkData.getTime(row));
     }
     timeStatistics.setEmpty(false);
@@ -110,7 +110,7 @@ public class AlignedReadOnlyMemChunk extends ReadOnlyMemChunk {
         valueStatistics.setEmpty(true);
         continue;
       }
-      for (int row = 0; row < alignedChunkData.size(); row++) {
+      for (int row = 0; row < alignedChunkData.rowCount(); row++) {
         long time = alignedChunkData.getTime(row);
         int originRowIndex = alignedChunkData.getValueIndex(row);
         boolean isNull = alignedChunkData.isValueMarked(originRowIndex, column);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 0f0d827..71eaef1 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -369,7 +369,7 @@ public class TsFileProcessor {
       } else {
         // here currentChunkPointNum >= 1
         long currentChunkPointNum =
-            workMemTable.getCurrentChunkPointNum(deviceID, insertRowPlan.getMeasurements()[i]);
+            workMemTable.getCurrentTVListSize(deviceID, insertRowPlan.getMeasurements()[i]);
         memTableIncrement +=
             (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE) == 0
                 ? TVList.tvListArrayMemCost(insertRowPlan.getDataTypes()[i])
@@ -409,7 +409,7 @@ public class TsFileProcessor {
     } else {
       // here currentChunkPointNum >= 1
       long currentChunkPointNum =
-          workMemTable.getCurrentChunkPointNum(deviceID, AlignedPath.VECTOR_PLACEHOLDER);
+          workMemTable.getCurrentTVListSize(deviceID, AlignedPath.VECTOR_PLACEHOLDER);
       memTableIncrement +=
           (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE) == 0
               ? AlignedTVList.alignedTvListArrayMemCost(insertRowPlan.getDataTypes())
@@ -518,7 +518,7 @@ public class TsFileProcessor {
           ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
               * TVList.tvListArrayMemCost(dataType);
     } else {
-      long currentChunkPointNum = workMemTable.getCurrentChunkPointNum(deviceId, measurement);
+      long currentChunkPointNum = workMemTable.getCurrentTVListSize(deviceId, measurement);
       if (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE == 0) {
         memIncrements[0] +=
             ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
@@ -558,7 +558,7 @@ public class TsFileProcessor {
               * AlignedTVList.alignedTvListArrayMemCost(dataTypes);
     } else {
       int currentChunkPointNum =
-          (int) workMemTable.getCurrentChunkPointNum(deviceId, AlignedPath.VECTOR_PLACEHOLDER);
+          (int) workMemTable.getCurrentTVListSize(deviceId, AlignedPath.VECTOR_PLACEHOLDER);
       if (currentChunkPointNum % PrimitiveArrayManager.ARRAY_SIZE == 0) {
         memIncrements[0] +=
             ((end - start) / PrimitiveArrayManager.ARRAY_SIZE + 1)
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
index 4c6c80d..9ae5b7d 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/path/AlignedPath.java
@@ -414,7 +414,7 @@ public class AlignedPath extends PartialPath {
     }
     // get sorted tv list is synchronized so different query can get right sorted list reference
     TVList alignedTvListCopy = alignedMemChunk.getSortedTvListForQuery(schemaList);
-    int curSize = alignedTvListCopy.size();
+    int curSize = alignedTvListCopy.rowCount();
     List<List<TimeRange>> deletionList = null;
     if (modsToMemtable != null) {
       deletionList = constructDeletionList(memTable, modsToMemtable, timeLowerBound);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java
index 5731ad5..7013cf7 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/path/MeasurementPath.java
@@ -269,7 +269,7 @@ public class MeasurementPath extends PartialPath {
     IWritableMemChunk memChunk = memTableMap.get(deviceID).getMemChunkMap().get(getMeasurement());
     // get sorted tv list is synchronized so different query can get right sorted list reference
     TVList chunkCopy = memChunk.getSortedTvListForQuery();
-    int curSize = chunkCopy.size();
+    int curSize = chunkCopy.rowCount();
     List<TimeRange> deletionList = null;
     if (modsToMemtable != null) {
       deletionList = constructDeletionList(memTable, modsToMemtable, timeLowerBound);
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java
index 6392ff9..c91348f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java
@@ -80,8 +80,8 @@ public class AlignedTVList extends TVList {
   @Override
   public void putAlignedValue(long timestamp, Object[] value, int[] columnIndexArray) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     for (int i = 0; i < values.size(); i++) {
@@ -119,9 +119,9 @@ public class AlignedTVList extends TVList {
           break;
       }
     }
-    indices.get(arrayIndex)[elementIndex] = size;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    indices.get(arrayIndex)[elementIndex] = rowCount;
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
@@ -139,7 +139,7 @@ public class AlignedTVList extends TVList {
 
   private Object getAlignedValueForQuery(
       int index, Integer floatPrecision, List<TSEncoding> encodingList) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -169,7 +169,7 @@ public class AlignedTVList extends TVList {
       int[] validIndexesForTimeDuplicatedRows,
       Integer floatPrecision,
       List<TSEncoding> encodingList) {
-    if (valueIndex >= size) {
+    if (valueIndex >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(valueIndex);
     }
     TsPrimitiveType[] vector = new TsPrimitiveType[values.size()];
@@ -261,7 +261,7 @@ public class AlignedTVList extends TVList {
     alignedTvList.indices = this.indices;
     alignedTvList.values = values;
     alignedTvList.bitMaps = bitMaps;
-    alignedTvList.size = this.size;
+    alignedTvList.rowCount = this.rowCount;
     return alignedTvList;
   }
 
@@ -300,7 +300,7 @@ public class AlignedTVList extends TVList {
       BitMap bitMap = new BitMap(ARRAY_SIZE);
       // last bitmap should be marked to the tslist size's position
       if (i == timestamps.size() - 1) {
-        for (int j = 0; j < size % ARRAY_SIZE; j++) {
+        for (int j = 0; j < rowCount % ARRAY_SIZE; j++) {
           bitMap.mark(j);
         }
       } else {
@@ -405,7 +405,7 @@ public class AlignedTVList extends TVList {
    * @return boolean
    */
   public boolean isValueMarked(int rowIndex, int columnIndex) {
-    if (rowIndex >= size) {
+    if (rowIndex >= rowCount) {
       return false;
     }
     if (bitMaps == null
@@ -447,7 +447,7 @@ public class AlignedTVList extends TVList {
   public Pair<Integer, Boolean> delete(long lowerBound, long upperBound, int columnIndex) {
     int deletedNumber = 0;
     boolean deleteColumn = true;
-    for (int i = 0; i < size; i++) {
+    for (int i = 0; i < rowCount; i++) {
       long time = getTime(i);
       if (time >= lowerBound && time <= upperBound) {
         int originRowIndex = getValueIndex(i);
@@ -557,14 +557,15 @@ public class AlignedTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedIndices == null || sortedIndices.length < size) {
-      sortedIndices = (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, size);
+    if (sortedIndices == null || sortedIndices.length < rowCount) {
+      sortedIndices =
+          (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -660,7 +661,7 @@ public class AlignedTVList extends TVList {
    */
   @Override
   public int getValueIndex(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -730,15 +731,15 @@ public class AlignedTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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);
         arrayCopy(value, idx, arrayIdx, elementIdx, inputRemaining, columnIndexArray);
         for (int i = 0; i < inputRemaining; i++) {
-          indices.get(arrayIdx)[elementIdx + i] = size;
+          indices.get(arrayIdx)[elementIdx + i] = rowCount;
           for (int j = 0; j < values.size(); j++) {
             if (columnIndexArray[j] < 0
                 || bitMaps != null
@@ -747,7 +748,7 @@ public class AlignedTVList extends TVList {
               markNullValue(j, arrayIdx, elementIdx + i);
             }
           }
-          size++;
+          rowCount++;
         }
         break;
       } else {
@@ -756,7 +757,7 @@ public class AlignedTVList extends TVList {
         System.arraycopy(time, idx, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         arrayCopy(value, idx, arrayIdx, elementIdx, internalRemaining, columnIndexArray);
         for (int i = 0; i < internalRemaining; i++) {
-          indices.get(arrayIdx)[elementIdx + i] = size;
+          indices.get(arrayIdx)[elementIdx + i] = rowCount;
           for (int j = 0; j < values.size(); j++) {
             if (columnIndexArray[j] < 0
                 || bitMaps != null
@@ -765,7 +766,7 @@ public class AlignedTVList extends TVList {
               markNullValue(j, arrayIdx, elementIdx + i);
             }
           }
-          size++;
+          rowCount++;
         }
         idx += internalRemaining;
         checkExpansion();
@@ -878,7 +879,7 @@ public class AlignedTVList extends TVList {
   }
 
   public void clear() {
-    size = 0;
+    rowCount = 0;
     sorted = true;
     minTime = Long.MAX_VALUE;
     clearTime();
@@ -941,7 +942,7 @@ public class AlignedTVList extends TVList {
       List<Integer> timeDuplicatedAlignedRowIndexList = null;
       while (cur < iteSize) {
         long time = getTime(cur);
-        if (cur + 1 < size() && (time == getTime(cur + 1))) {
+        if (cur + 1 < rowCount() && (time == getTime(cur + 1))) {
           if (timeDuplicatedAlignedRowIndexList == null) {
             timeDuplicatedAlignedRowIndexList = new ArrayList<>();
             timeDuplicatedAlignedRowIndexList.add(getValueIndex(cur));
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 136c283..a365862 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
@@ -49,20 +49,20 @@ public class BinaryTVList extends TVList {
   @Override
   public void putBinary(long timestamp, Binary value) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     values.get(arrayIndex)[elementIndex] = value;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
 
   @Override
   public Binary getBinary(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -71,7 +71,7 @@ public class BinaryTVList extends TVList {
   }
 
   protected void set(int index, long timestamp, Binary value) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -98,15 +98,15 @@ public class BinaryTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedValues == null || sortedValues.length < size) {
+    if (sortedValues == null || sortedValues.length < rowCount) {
       sortedValues =
-          (Binary[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.TEXT, size);
+          (Binary[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.TEXT, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -219,15 +219,15 @@ public class BinaryTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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 - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
+        rowCount += inputRemaining;
         break;
       } else {
         // the remaining inputs cannot fit the last array, fill the last array and create a new
@@ -236,7 +236,7 @@ public class BinaryTVList extends TVList {
             time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
         idx += internalRemaining;
-        size += internalRemaining;
+        rowCount += internalRemaining;
         checkExpansion();
       }
     }
@@ -268,7 +268,7 @@ public class BinaryTVList extends TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
     return nullCnt;
   }
 
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 f4c9b31..390c9b6 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
@@ -48,20 +48,20 @@ public class BooleanTVList extends TVList {
   @Override
   public void putBoolean(long timestamp, boolean value) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     values.get(arrayIndex)[elementIndex] = value;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
 
   @Override
   public boolean getBoolean(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -70,7 +70,7 @@ public class BooleanTVList extends TVList {
   }
 
   protected void set(int index, long timestamp, boolean value) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -97,15 +97,15 @@ public class BooleanTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedValues == null || sortedValues.length < size) {
+    if (sortedValues == null || sortedValues.length < rowCount) {
       sortedValues =
-          (boolean[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.BOOLEAN, size);
+          (boolean[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.BOOLEAN, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -219,15 +219,15 @@ public class BooleanTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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 - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
+        rowCount += inputRemaining;
         break;
       } else {
         // the remaining inputs cannot fit the last array, fill the last array and create a new
@@ -236,7 +236,7 @@ public class BooleanTVList extends TVList {
             time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
         idx += internalRemaining;
-        size += internalRemaining;
+        rowCount += internalRemaining;
         checkExpansion();
       }
     }
@@ -268,7 +268,7 @@ public class BooleanTVList extends TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
     return nullCnt;
   }
 
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 6b3fcb8..a7b6718 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
@@ -49,20 +49,20 @@ public class DoubleTVList extends TVList {
   @Override
   public void putDouble(long timestamp, double value) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     values.get(arrayIndex)[elementIndex] = value;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
 
   @Override
   public double getDouble(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -71,7 +71,7 @@ public class DoubleTVList extends TVList {
   }
 
   protected void set(int index, long timestamp, double value) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -98,15 +98,15 @@ public class DoubleTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedValues == null || sortedValues.length < size) {
+    if (sortedValues == null || sortedValues.length < rowCount) {
       sortedValues =
-          (double[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.DOUBLE, size);
+          (double[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.DOUBLE, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -223,15 +223,15 @@ public class DoubleTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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 - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
+        rowCount += inputRemaining;
         break;
       } else {
         // the remaining inputs cannot fit the last array, fill the last array and create a new
@@ -240,7 +240,7 @@ public class DoubleTVList extends TVList {
             time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
         idx += internalRemaining;
-        size += internalRemaining;
+        rowCount += internalRemaining;
         checkExpansion();
       }
     }
@@ -272,7 +272,7 @@ public class DoubleTVList extends TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
     return nullCnt;
   }
 
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 3e4078a..8594067 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
@@ -49,20 +49,20 @@ public class FloatTVList extends TVList {
   @Override
   public void putFloat(long timestamp, float value) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     values.get(arrayIndex)[elementIndex] = value;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
 
   @Override
   public float getFloat(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -71,7 +71,7 @@ public class FloatTVList extends TVList {
   }
 
   protected void set(int index, long timestamp, float value) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -98,15 +98,15 @@ public class FloatTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedValues == null || sortedValues.length < size) {
+    if (sortedValues == null || sortedValues.length < rowCount) {
       sortedValues =
-          (float[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.FLOAT, size);
+          (float[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.FLOAT, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -223,15 +223,15 @@ public class FloatTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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 - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
+        rowCount += inputRemaining;
         break;
       } else {
         // the remaining inputs cannot fit the last array, fill the last array and create a new
@@ -240,7 +240,7 @@ public class FloatTVList extends TVList {
             time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
         idx += internalRemaining;
-        size += internalRemaining;
+        rowCount += internalRemaining;
         checkExpansion();
       }
     }
@@ -272,7 +272,7 @@ public class FloatTVList extends TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
     return nullCnt;
   }
 
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 7a7f35a..8f06aa0 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
@@ -48,20 +48,20 @@ public class IntTVList extends TVList {
   @Override
   public void putInt(long timestamp, int value) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     values.get(arrayIndex)[elementIndex] = value;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
 
   @Override
   public int getInt(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -70,7 +70,7 @@ public class IntTVList extends TVList {
   }
 
   protected void set(int index, long timestamp, int value) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -97,14 +97,15 @@ public class IntTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, size);
+    if (sortedValues == null || sortedValues.length < rowCount) {
+      sortedValues =
+          (int[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT32, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -217,15 +218,15 @@ public class IntTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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 - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
+        rowCount += inputRemaining;
         break;
       } else {
         // the remaining inputs cannot fit the last array, fill the last array and create a new
@@ -234,7 +235,7 @@ public class IntTVList extends TVList {
             time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
         idx += internalRemaining;
-        size += internalRemaining;
+        rowCount += internalRemaining;
         checkExpansion();
       }
     }
@@ -266,7 +267,7 @@ public class IntTVList extends TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
     return nullCnt;
   }
 
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 c8c8ddd..1cb1e60 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
@@ -48,20 +48,20 @@ public class LongTVList extends TVList {
   @Override
   public void putLong(long timestamp, long value) {
     checkExpansion();
-    int arrayIndex = size / ARRAY_SIZE;
-    int elementIndex = size % ARRAY_SIZE;
+    int arrayIndex = rowCount / ARRAY_SIZE;
+    int elementIndex = rowCount % ARRAY_SIZE;
     minTime = Math.min(minTime, timestamp);
     timestamps.get(arrayIndex)[elementIndex] = timestamp;
     values.get(arrayIndex)[elementIndex] = value;
-    size++;
-    if (sorted && size > 1 && timestamp < getTime(size - 2)) {
+    rowCount++;
+    if (sorted && rowCount > 1 && timestamp < getTime(rowCount - 2)) {
       sorted = false;
     }
   }
 
   @Override
   public long getLong(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -70,7 +70,7 @@ public class LongTVList extends TVList {
   }
 
   protected void set(int index, long timestamp, long value) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -97,14 +97,15 @@ public class LongTVList extends TVList {
 
   @Override
   public void sort() {
-    if (sortedTimestamps == null || sortedTimestamps.length < size) {
+    if (sortedTimestamps == null || sortedTimestamps.length < rowCount) {
       sortedTimestamps =
-          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    if (sortedValues == null || sortedValues.length < size) {
-      sortedValues = (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, size);
+    if (sortedValues == null || sortedValues.length < rowCount) {
+      sortedValues =
+          (long[][]) PrimitiveArrayManager.createDataListsByType(TSDataType.INT64, rowCount);
     }
-    sort(0, size);
+    sort(0, rowCount);
     clearSortedValue();
     clearSortedTime();
     sorted = true;
@@ -217,15 +218,15 @@ public class LongTVList extends TVList {
 
     while (idx < end) {
       int inputRemaining = end - idx;
-      int arrayIdx = size / ARRAY_SIZE;
-      int elementIdx = size % ARRAY_SIZE;
+      int arrayIdx = rowCount / ARRAY_SIZE;
+      int elementIdx = rowCount % 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 - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, inputRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, inputRemaining);
-        size += inputRemaining;
+        rowCount += inputRemaining;
         break;
       } else {
         // the remaining inputs cannot fit the last array, fill the last array and create a new
@@ -234,7 +235,7 @@ public class LongTVList extends TVList {
             time, idx - timeIdxOffset, timestamps.get(arrayIdx), elementIdx, internalRemaining);
         System.arraycopy(value, idx, values.get(arrayIdx), elementIdx, internalRemaining);
         idx += internalRemaining;
-        size += internalRemaining;
+        rowCount += internalRemaining;
         checkExpansion();
       }
     }
@@ -266,7 +267,7 @@ public class LongTVList extends TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
     return nullCnt;
   }
 
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 ec383ea..1f9c1e7 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
@@ -45,7 +45,7 @@ public abstract class TVList {
   // list of timestamp array, add 1 when expanded -> data point timestamp array
   // index relation: arrayIndex -> elementIndex
   protected List<long[]> timestamps;
-  protected int size;
+  protected int rowCount;
 
   protected long[][] sortedTimestamps;
   protected boolean sorted = true;
@@ -59,7 +59,7 @@ public abstract class TVList {
 
   public TVList() {
     timestamps = new ArrayList<>();
-    size = 0;
+    rowCount = 0;
     minTime = Long.MAX_VALUE;
     referenceCount = new AtomicInteger();
   }
@@ -109,12 +109,12 @@ public abstract class TVList {
     return referenceCount.get();
   }
 
-  public int size() {
-    return size;
+  public int rowCount() {
+    return rowCount;
   }
 
   public long getTime(int index) {
-    if (index >= size) {
+    if (index >= rowCount) {
       throw new ArrayIndexOutOfBoundsException(index);
     }
     int arrayIndex = index / ARRAY_SIZE;
@@ -256,15 +256,15 @@ public abstract class TVList {
   public int delete(long lowerBound, long upperBound) {
     int newSize = 0;
     minTime = Long.MAX_VALUE;
-    for (int i = 0; i < size; i++) {
+    for (int i = 0; i < rowCount; i++) {
       long time = getTime(i);
       if (time < lowerBound || time > upperBound) {
         set(i, newSize++);
         minTime = Math.min(time, minTime);
       }
     }
-    int deletedNumber = size - newSize;
-    size = newSize;
+    int deletedNumber = rowCount - newSize;
+    rowCount = newSize;
     // release primitive arrays that are empty
     int newArrayNum = newSize / ARRAY_SIZE;
     if (newSize % ARRAY_SIZE != 0) {
@@ -282,13 +282,13 @@ public abstract class TVList {
     for (long[] timestampArray : timestamps) {
       cloneList.timestamps.add(cloneTime(timestampArray));
     }
-    cloneList.size = size;
+    cloneList.rowCount = rowCount;
     cloneList.sorted = sorted;
     cloneList.minTime = minTime;
   }
 
   public void clear() {
-    size = 0;
+    rowCount = 0;
     sorted = true;
     minTime = Long.MAX_VALUE;
     clearTime();
@@ -322,7 +322,7 @@ public abstract class TVList {
   abstract void clearSortedValue();
 
   protected void checkExpansion() {
-    if ((size % ARRAY_SIZE) == 0) {
+    if ((rowCount % ARRAY_SIZE) == 0) {
       expandValues();
       timestamps.add((long[]) getPrimitiveArraysByType(TSDataType.INT64));
     }
@@ -501,7 +501,7 @@ public abstract class TVList {
       }
     }
     minTime = Math.min(inPutMinTime, minTime);
-    sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
+    sorted = sorted && inputSorted && (rowCount == 0 || inPutMinTime >= getTime(rowCount - 1));
   }
 
   /** for log */
@@ -544,7 +544,7 @@ public abstract class TVList {
     private List<TimeRange> deletionList;
 
     public Ite() {
-      this.iteSize = TVList.this.size;
+      this.iteSize = TVList.this.rowCount;
     }
 
     public Ite(int floatPrecision, TSEncoding encoding, int size, List<TimeRange> deletionList) {
@@ -562,7 +562,7 @@ public abstract class TVList {
 
       while (cur < iteSize) {
         long time = getTime(cur);
-        if (isPointDeleted(time) || (cur + 1 < size() && (time == getTime(cur + 1)))) {
+        if (isPointDeleted(time) || (cur + 1 < rowCount() && (time == getTime(cur + 1)))) {
           cur++;
           continue;
         }
@@ -616,6 +616,6 @@ public abstract class TVList {
   public abstract TSDataType getDataType();
 
   public long getLastTime() {
-    return getTime(size - 1);
+    return getTime(rowCount - 1);
   }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index 444de28..2341b47 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@ -302,11 +302,41 @@ public class TsFileProcessorTest {
     processor.setTsFileProcessorInfo(tsFileProcessorInfo);
     this.sgInfo.initTsFileProcessorInfo(processor);
     SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    processor.insertTablet(genInsertTablePlan(0), 0, 100, new TSStatus[100]);
+    processor.insertTablet(genInsertTablePlan(0, true), 0, 10, new TSStatus[10]);
     IMemTable memTable = processor.getWorkMemTable();
-    Assert.assertEquals(3008, memTable.getTVListsRamCost());
-    processor.insertTablet(genInsertTablePlan(100), 0, 100, new TSStatus[100]);
-    Assert.assertEquals(5264, memTable.getTVListsRamCost());
+    Assert.assertEquals(828424, memTable.getTVListsRamCost());
+    processor.insertTablet(genInsertTablePlan(100, true), 0, 10, new TSStatus[10]);
+    Assert.assertEquals(828424, memTable.getTVListsRamCost());
+    processor.insertTablet(genInsertTablePlan(200, true), 0, 10, new TSStatus[10]);
+    Assert.assertEquals(828424, memTable.getTVListsRamCost());
+    Assert.assertEquals(90000, memTable.getTotalPointsNum());
+    Assert.assertEquals(720360, memTable.memSize());
+  }
+
+  @Test
+  public void nonAlignedTvListRamCostTest()
+      throws MetadataException, WriteProcessException, IOException {
+    processor =
+        new TsFileProcessor(
+            storageGroup,
+            SystemFileFactory.INSTANCE.getFile(filePath),
+            sgInfo,
+            this::closeTsFileProcessor,
+            (tsFileProcessor) -> true,
+            true);
+    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
+    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
+    this.sgInfo.initTsFileProcessorInfo(processor);
+    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
+    processor.insertTablet(genInsertTablePlan(0, false), 0, 10, new TSStatus[10]);
+    IMemTable memTable = processor.getWorkMemTable();
+    Assert.assertEquals(1656000, memTable.getTVListsRamCost());
+    processor.insertTablet(genInsertTablePlan(100, false), 0, 10, new TSStatus[10]);
+    Assert.assertEquals(1656000, memTable.getTVListsRamCost());
+    processor.insertTablet(genInsertTablePlan(200, false), 0, 10, new TSStatus[10]);
+    Assert.assertEquals(1656000, memTable.getTVListsRamCost());
+    Assert.assertEquals(90000, memTable.getTotalPointsNum());
+    Assert.assertEquals(1440000, memTable.memSize());
   }
 
   @Test
@@ -384,50 +414,45 @@ public class TsFileProcessorTest {
     }
   }
 
-  private InsertTabletPlan genInsertTablePlan(long startTime) throws IllegalPathException {
-    String[] measurements = new String[2];
-    measurements[0] = "sensor0";
-    measurements[1] = "sensor1";
-
-    List<Integer> dataTypesList = new ArrayList<>();
-    TSDataType[] dataTypes = new TSDataType[2];
-    dataTypesList.add(TSDataType.BOOLEAN.ordinal());
-    dataTypesList.add(TSDataType.INT64.ordinal());
-    dataTypes[0] = TSDataType.BOOLEAN;
-    dataTypes[1] = TSDataType.INT64;
-
-    TSEncoding[] encodings = new TSEncoding[2];
-    encodings[0] = TSEncoding.PLAIN;
-    encodings[1] = TSEncoding.GORILLA;
-
+  private InsertTabletPlan genInsertTablePlan(long startTime, boolean isAligned)
+      throws IllegalPathException {
     String deviceId = "root.sg.device5";
-
-    IMeasurementMNode[] mNodes = new IMeasurementMNode[2];
-    IMeasurementSchema schema0 = new MeasurementSchema(measurements[0], dataTypes[0], encodings[0]);
-    IMeasurementSchema schema1 = new MeasurementSchema(measurements[1], dataTypes[1], encodings[1]);
-    mNodes[0] = MeasurementMNode.getMeasurementMNode(null, "sensor0", schema0, null);
-    mNodes[1] = MeasurementMNode.getMeasurementMNode(null, "sensor1", schema1, null);
-
+    String[] measurements = new String[3000];
+    List<Integer> dataTypesList = new ArrayList<>();
+    TSDataType[] dataTypes = new TSDataType[3000];
+    TSEncoding[] encodings = new TSEncoding[3000];
+    IMeasurementMNode[] mNodes = new IMeasurementMNode[3000];
+    for (int i = 0; i < 3000; i++) {
+      measurements[i] = "s" + i;
+      dataTypesList.add(TSDataType.INT64.ordinal());
+      dataTypes[i] = TSDataType.INT64;
+      encodings[i] = TSEncoding.PLAIN;
+      IMeasurementSchema schema =
+          new MeasurementSchema(measurements[i], dataTypes[i], encodings[i]);
+      mNodes[i] = MeasurementMNode.getMeasurementMNode(null, measurements[i], schema, null);
+    }
     InsertTabletPlan insertTabletPlan =
         new InsertTabletPlan(new PartialPath(deviceId), measurements, dataTypesList);
 
-    long[] times = new long[100];
-    Object[] columns = new Object[2];
-    columns[0] = new boolean[100];
-    columns[1] = new long[100];
+    long[] times = new long[10];
+    Object[] columns = new Object[3000];
+    for (int i = 0; i < 3000; i++) {
+      columns[i] = new long[10];
+    }
 
-    for (long r = 0; r < 100; r++) {
+    for (long r = 0; r < 10; r++) {
       times[(int) r] = r + startTime;
-      ((boolean[]) columns[0])[(int) r] = false;
-      ((long[]) columns[1])[(int) r] = r;
+      for (int i = 0; i < 3000; i++) {
+        ((long[]) columns[i])[(int) r] = r;
+      }
     }
     insertTabletPlan.setTimes(times);
     insertTabletPlan.setColumns(columns);
     insertTabletPlan.setRowCount(times.length);
     insertTabletPlan.setMeasurementMNodes(mNodes);
     insertTabletPlan.setStart(0);
-    insertTabletPlan.setEnd(100);
-    insertTabletPlan.setAligned(true);
+    insertTabletPlan.setEnd(10);
+    insertTabletPlan.setAligned(isAligned);
 
     return insertTabletPlan;
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java
index 8e09c7d..4621b10 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BinaryTVListTest.java
@@ -36,7 +36,7 @@ public class BinaryTVListTest {
     for (int i = 0; i < 1000; i++) {
       tvList.putBinary(i, Binary.valueOf(String.valueOf(i)));
     }
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(String.valueOf(i), tvList.getBinary(i).toString());
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -53,8 +53,8 @@ public class BinaryTVListTest {
     }
     tvList.putBinaries(
         ArrayUtils.toPrimitive(timeList.toArray(new Long[0])), binaryList, null, 0, 1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -103,7 +103,7 @@ public class BinaryTVListTest {
         ArrayUtils.toPrimitive(timeList.toArray(new Long[0])), binaryList, bitMap, 0, 1000);
     tvList.sort();
     BinaryTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getBinary((int) i), clonedTvList.getBinary((int) i));
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BooleanTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BooleanTVListTest.java
index 09eb4ab..ecc1ece 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BooleanTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/BooleanTVListTest.java
@@ -37,11 +37,11 @@ public class BooleanTVListTest {
     for (int i = 50; i < 100; i++) {
       tvList.putBoolean(i, false);
     }
-    for (int i = 0; i < tvList.size / 2; i++) {
+    for (int i = 0; i < tvList.rowCount / 2; i++) {
       Assert.assertTrue(tvList.getBoolean(i));
       Assert.assertEquals(i, tvList.getTime(i));
     }
-    for (int i = tvList.size / 2 + 1; i < tvList.size; i++) {
+    for (int i = tvList.rowCount / 2 + 1; i < tvList.rowCount; i++) {
       Assert.assertFalse(tvList.getBoolean(i));
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -62,8 +62,8 @@ public class BooleanTVListTest {
         null,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -119,7 +119,7 @@ public class BooleanTVListTest {
         1000);
     tvList.sort();
     BooleanTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getBoolean((int) i), clonedTvList.getBoolean((int) i));
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/DoubleTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/DoubleTVListTest.java
index 315c0a3..1f35cb4 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/DoubleTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/DoubleTVListTest.java
@@ -37,7 +37,7 @@ public class DoubleTVListTest {
       tvList.putDouble(i, i);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(i, tvList.getDouble(i), delta);
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -50,7 +50,7 @@ public class DoubleTVListTest {
       tvList.putDouble(i, i);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(i, tvList.getDouble(i), delta);
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -71,9 +71,9 @@ public class DoubleTVListTest {
         null,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals((double) tvList.size - i, tvList.getDouble((int) i), delta);
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals((double) tvList.rowCount - i, tvList.getDouble((int) i), delta);
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -129,7 +129,7 @@ public class DoubleTVListTest {
         1000);
     tvList.sort();
     DoubleTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getDouble((int) i), clonedTvList.getDouble((int) i), delta);
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/FloatTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/FloatTVListTest.java
index 90c1c12..9299af5 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/FloatTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/FloatTVListTest.java
@@ -37,7 +37,7 @@ public class FloatTVListTest {
       tvList.putFloat(i, (float) i);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals((float) i, tvList.getFloat(i), delta);
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -50,7 +50,7 @@ public class FloatTVListTest {
       tvList.putFloat(i, (float) i);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals((float) i, tvList.getFloat(i), delta);
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -71,9 +71,9 @@ public class FloatTVListTest {
         null,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals((float) tvList.size - i, tvList.getFloat((int) i), delta);
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals((float) tvList.rowCount - i, tvList.getFloat((int) i), delta);
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -129,7 +129,7 @@ public class FloatTVListTest {
         1000);
     tvList.sort();
     FloatTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getFloat((int) i), clonedTvList.getFloat((int) i), delta);
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/IntTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/IntTVListTest.java
index 8384b80..bfc45fa 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/IntTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/IntTVListTest.java
@@ -36,7 +36,7 @@ public class IntTVListTest {
       tvList.putInt(i, i);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(i, tvList.getInt(i));
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -49,7 +49,7 @@ public class IntTVListTest {
       tvList.putInt(i, i);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(i, tvList.getInt(i));
       Assert.assertEquals(i, tvList.getTime(i));
     }
@@ -70,9 +70,9 @@ public class IntTVListTest {
         null,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals(tvList.size - i, tvList.getInt((int) i));
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals(tvList.rowCount - i, tvList.getInt((int) i));
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -128,7 +128,7 @@ public class IntTVListTest {
         1000);
     tvList.sort();
     IntTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getInt((int) i), clonedTvList.getInt((int) i));
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java
index 4af8cb7..401b488 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/LongTVListTest.java
@@ -39,7 +39,7 @@ public class LongTVListTest {
       tvList.putLong(i, i);
     }
     tvList.sort();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(i, tvList.getLong((int) i));
       Assert.assertEquals(i, tvList.getTime((int) i));
     }
@@ -52,7 +52,7 @@ public class LongTVListTest {
       tvList.putLong(i, i);
     }
     tvList.sort();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(i, tvList.getLong((int) i));
       Assert.assertEquals(i, tvList.getTime((int) i));
     }
@@ -71,7 +71,7 @@ public class LongTVListTest {
     }
     tvList.sort();
     inputs.sort(TimeValuePair::compareTo);
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(inputs.get((int) i).getTimestamp(), tvList.getTime((int) i));
       Assert.assertEquals(inputs.get((int) i).getValue().getLong(), tvList.getLong((int) i));
     }
@@ -92,9 +92,9 @@ public class LongTVListTest {
         null,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals(tvList.size - i, tvList.getLong((int) i));
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals(tvList.rowCount - i, tvList.getLong((int) i));
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -150,7 +150,7 @@ public class LongTVListTest {
         1000);
     tvList.sort();
     LongTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getLong((int) i), clonedTvList.getLong((int) i));
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java
index 7a6d9aa..d9c7ff3 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/datastructure/VectorTVListTest.java
@@ -47,7 +47,7 @@ public class VectorTVListTest {
       }
       tvList.putAlignedValue(i, value, columnOrder);
     }
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       StringBuilder builder = new StringBuilder("[");
       builder.append(String.valueOf(i));
       for (int j = 1; j < 5; j++) {
@@ -84,7 +84,7 @@ public class VectorTVListTest {
       tvList.putAlignedValue(i, value, columnOrder);
     }
     tvList.sort();
-    for (int i = 0; i < tvList.size; i++) {
+    for (int i = 0; i < tvList.rowCount; i++) {
       StringBuilder builder = new StringBuilder("[");
       builder.append("false, 100, 1000, 0.1, 0.2, Test");
       builder.append("]");
@@ -118,8 +118,8 @@ public class VectorTVListTest {
         columnOrder,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
     }
   }
 
@@ -153,8 +153,8 @@ public class VectorTVListTest {
         columnOrder,
         0,
         1000);
-    for (long i = 0; i < tvList.size; i++) {
-      Assert.assertEquals(tvList.size - i, tvList.getTime((int) i));
+    for (long i = 0; i < tvList.rowCount; i++) {
+      Assert.assertEquals(tvList.rowCount - i, tvList.getTime((int) i));
       if (i % 100 == 0) {
         Assert.assertEquals(
             "[null, null, null, null, null]", tvList.getAlignedValue((int) i).toString());
@@ -194,7 +194,7 @@ public class VectorTVListTest {
         1000);
 
     AlignedTVList clonedTvList = tvList.clone();
-    for (long i = 0; i < tvList.size; i++) {
+    for (long i = 0; i < tvList.rowCount; i++) {
       Assert.assertEquals(tvList.getTime((int) i), clonedTvList.getTime((int) i));
       Assert.assertEquals(
           tvList.getAlignedValue((int) i).toString(),