You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by xi...@apache.org on 2023/01/05 03:40:29 UTC

[iotdb] branch iotdb5308-1.0 created (now 36cd717b30)

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

xiangweiwei pushed a change to branch iotdb5308-1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at 36cd717b30 Turn down the max capacity of tsblock from scanOperator (#8599)

This branch includes the following new commits:

     new 0c54cd1ae9 [IOTDB-5308] Limit the output tsblock's size of each operator[Part1]
     new 36cd717b30 Turn down the max capacity of tsblock from scanOperator (#8599)

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



[iotdb] 02/02: Turn down the max capacity of tsblock from scanOperator (#8599)

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

xiangweiwei pushed a commit to branch iotdb5308-1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 36cd717b300d9b0f31a5da2cda3375648c2595fa
Author: Xiangwei Wei <34...@users.noreply.github.com>
AuthorDate: Fri Dec 23 23:53:51 2022 +0800

    Turn down the max capacity of tsblock from scanOperator (#8599)
    
    (cherry picked from commit 424528495c5515c314f542d2523f7c0431ddc902)
---
 .../src/assembly/resources/conf/iotdb-common.properties      |  2 +-
 .../org/apache/iotdb/db/mpp/execution/memory/MemoryPool.java | 12 ++++++++++++
 .../iotdb/db/mpp/execution/exchange/SinkHandleTest.java      |  4 ++--
 .../iotdb/db/mpp/execution/operator/OperatorMemoryTest.java  | 10 ++++++----
 .../org/apache/iotdb/tsfile/common/conf/TSFileConfig.java    |  4 ++--
 .../iotdb/tsfile/read/common/block/TsBlockBuilder.java       |  2 +-
 6 files changed, 24 insertions(+), 10 deletions(-)

diff --git a/node-commons/src/assembly/resources/conf/iotdb-common.properties b/node-commons/src/assembly/resources/conf/iotdb-common.properties
index 92c4eb73f0..ee21625cfd 100644
--- a/node-commons/src/assembly/resources/conf/iotdb-common.properties
+++ b/node-commons/src/assembly/resources/conf/iotdb-common.properties
@@ -405,7 +405,7 @@ cluster_name=defaultCluster
 
 # The max capacity of a TsBlock
 # Datatype: int, Unit: byte
-# max_tsblock_size_in_bytes=1048576
+# max_tsblock_size_in_bytes=131072
 
 # The max number of lines in a single TsBlock
 # Datatype: int
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/memory/MemoryPool.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/memory/MemoryPool.java
index b247846e59..b3d2fc2ff8 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/memory/MemoryPool.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/memory/MemoryPool.java
@@ -154,6 +154,14 @@ public class MemoryPool {
         bytesToReserve > 0L && bytesToReserve <= maxBytesPerFragmentInstance,
         "bytes should be greater than zero while less than or equal to max bytes per fragment instance: %d",
         bytesToReserve);
+    if (bytesToReserve > maxBytesCanReserve) {
+      LOGGER.warn(
+          "Cannot reserve {} bytes memory from MemoryPool for planNodeId{}",
+          bytesToReserve,
+          planNodeId);
+      throw new IllegalArgumentException(
+          "Query is aborted since it requests more memory than can be allocated.");
+    }
 
     ListenableFuture<Void> result;
     synchronized (this) {
@@ -164,6 +172,10 @@ public class MemoryPool {
                       .getOrDefault(fragmentInstanceId, Collections.emptyMap())
                       .getOrDefault(planNodeId, 0L)
               < bytesToReserve) {
+        LOGGER.debug(
+            "Blocked reserve request: {} bytes memory for planNodeId{}",
+            bytesToReserve,
+            planNodeId);
         result =
             MemoryReservationFuture.create(
                 queryId, fragmentInstanceId, planNodeId, bytesToReserve, maxBytesCanReserve);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandleTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandleTest.java
index 57f121f40f..0c8ebf9f73 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandleTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandleTest.java
@@ -49,7 +49,7 @@ public class SinkHandleTest {
   @Test
   public void testOneTimeNotBlockedSend() {
     final String queryId = "q0";
-    final long mockTsBlockSize = 1024L * 1024L;
+    final long mockTsBlockSize = 128 * 1024L;
     final int numOfMockTsBlock = 1;
     final TEndPoint remoteEndpoint =
         new TEndPoint("remote", IoTDBDescriptor.getInstance().getConfig().getMppDataExchangePort());
@@ -187,7 +187,7 @@ public class SinkHandleTest {
   @Test
   public void testMultiTimesBlockedSend() {
     final String queryId = "q0";
-    final long mockTsBlockSize = 1024L * 1024L;
+    final long mockTsBlockSize = 128 * 1024L;
     final int numOfMockTsBlock = 1;
     final TEndPoint remoteEndpoint =
         new TEndPoint("remote", IoTDBDescriptor.getInstance().getConfig().getMppDataExchangePort());
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
index 2652173e47..8bc9f944a4 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
@@ -1261,10 +1261,12 @@ public class OperatorMemoryTest {
               typeProvider);
 
       expectedMaxReturnSize =
-          maxTsBlockLineNumber
-              * (TimeColumn.SIZE_IN_BYTES_PER_POSITION
-                  + 512 * Byte.BYTES
-                  + LongColumn.SIZE_IN_BYTES_PER_POSITION);
+          Math.min(
+              DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES,
+              maxTsBlockLineNumber
+                  * (TimeColumn.SIZE_IN_BYTES_PER_POSITION
+                      + 512 * Byte.BYTES
+                      + LongColumn.SIZE_IN_BYTES_PER_POSITION));
       expectedMaxRetainSize = 2L * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
 
       assertEquals(
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
index 6dbd28a67e..a1a5b22f88 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
@@ -151,8 +151,8 @@ public class TSFileConfig implements Serializable {
   /** The amount of data iterate each time */
   private int batchSize = 1000;
 
-  /** Maximum capacity of a TsBlock */
-  private int maxTsBlockSizeInBytes = 1024 * 1024;
+  /** Maximum capacity of a TsBlock, allow up to two pages. */
+  private int maxTsBlockSizeInBytes = 128 * 1024;
 
   /** Maximum number of lines in a single TsBlock */
   private int maxTsBlockLineNumber = 1000;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java
index d4152d4ddb..c309835a09 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java
@@ -259,7 +259,7 @@ public class TsBlockBuilder {
   }
 
   public boolean isFull() {
-    return declaredPositions == MAX_LINE_NUMBER || tsBlockBuilderStatus.isFull();
+    return declaredPositions >= MAX_LINE_NUMBER || tsBlockBuilderStatus.isFull();
   }
 
   public boolean isEmpty() {


[iotdb] 01/02: [IOTDB-5308] Limit the output tsblock's size of each operator[Part1]

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

xiangweiwei pushed a commit to branch iotdb5308-1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 0c54cd1ae9600c6c4334cbaea8923376219e5fba
Author: Xiangwei Wei <34...@users.noreply.github.com>
AuthorDate: Thu Jan 5 10:10:32 2023 +0800

    [IOTDB-5308] Limit the output tsblock's size of each operator[Part1]
    
    (cherry picked from commit df78367ee6ac6a83f6c61a032bb2718b86126057)
---
 .../apache/iotdb/db/it/IoTDBFloatPrecisionIT.java  |  3 -
 .../mpp/execution/operator/AbstractOperator.java   | 94 ++++++++++++++++++++++
 .../operator/process/AbstractProcessOperator.java  | 24 ++++++
 .../process/join/RowBasedTimeJoinOperator.java     | 35 ++++----
 .../operator/process/join/TimeJoinOperator.java    | 31 +++----
 .../source/AbstractDataSourceOperator.java         | 32 ++++++++
 .../AbstractSeriesAggregationScanOperator.java     | 21 +----
 .../operator/source/AbstractSourceOperator.java    | 33 ++++++++
 .../operator/source/AlignedSeriesScanOperator.java | 50 +++++-------
 .../operator/source/SeriesScanOperator.java        | 41 +++-------
 .../mpp/execution/operator/OperatorMemoryTest.java | 28 ++++---
 .../iotdb/tsfile/read/common/block/TsBlock.java    |  9 +++
 .../read/common/block/column/BinaryColumn.java     |  5 ++
 .../read/common/block/column/BooleanColumn.java    |  5 ++
 .../tsfile/read/common/block/column/Column.java    |  2 +
 .../read/common/block/column/DoubleColumn.java     |  5 ++
 .../read/common/block/column/FloatColumn.java      |  5 ++
 .../tsfile/read/common/block/column/IntColumn.java |  5 ++
 .../read/common/block/column/LongColumn.java       |  5 ++
 .../read/common/block/column/NullColumn.java       |  5 ++
 .../block/column/RunLengthEncodedColumn.java       |  5 ++
 .../read/common/block/column/TimeColumn.java       |  5 ++
 22 files changed, 330 insertions(+), 118 deletions(-)

diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFloatPrecisionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFloatPrecisionIT.java
index 8c85dbbcba..b9cb021564 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFloatPrecisionIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFloatPrecisionIT.java
@@ -42,7 +42,6 @@ import java.util.List;
 import static org.apache.iotdb.db.constant.TestConstant.TIMESTAMP_STR;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
@@ -108,8 +107,6 @@ public class IoTDBFloatPrecisionIT {
   public void selectAllSQLTest() {
     try (Connection connection = EnvFactory.getEnv().getConnection();
         Statement statement = connection.createStatement()) {
-      boolean hasResultSet = statement.execute("select * from root.**");
-      assertTrue(hasResultSet);
       int cnt;
       try (ResultSet resultSet = statement.executeQuery("select * from root.**")) {
         assertNotNull(resultSet);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/AbstractOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/AbstractOperator.java
new file mode 100644
index 0000000000..8e5dc25271
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/AbstractOperator.java
@@ -0,0 +1,94 @@
+/*
+ * 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.mpp.execution.operator;
+
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.read.common.block.TsBlock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractOperator implements Operator {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(AbstractOperator.class);
+  protected OperatorContext operatorContext;
+
+  protected long maxReturnSize =
+      TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes();
+  protected int maxTupleSizeOfTsBlock = -1;
+  protected TsBlock resultTsBlock;
+  protected TsBlock retainedTsBlock;
+  protected int startOffset = 0;
+
+  public void initializeMaxTsBlockLength(TsBlock tsBlock) {
+    if (maxTupleSizeOfTsBlock != -1) {
+      return;
+    }
+    long oneTupleSize =
+        (tsBlock.getRetainedSizeInBytes() - tsBlock.getTotalInstanceSize())
+            / tsBlock.getPositionCount();
+    if (oneTupleSize > maxReturnSize) {
+      // make sure at least one-tuple-at-a-time
+      this.maxTupleSizeOfTsBlock = 1;
+      LOGGER.warn("Only one tuple can be sent each time caused by limited memory");
+    } else {
+      this.maxTupleSizeOfTsBlock = (int) (maxReturnSize / oneTupleSize);
+    }
+    LOGGER.debug("maxTupleSizeOfTsBlock is:{}", maxTupleSizeOfTsBlock);
+  }
+
+  public TsBlock checkTsBlockSizeAndGetResult() {
+    if (resultTsBlock == null) {
+      throw new IllegalArgumentException("Result tsBlock cannot be null");
+    }
+    if (maxTupleSizeOfTsBlock == -1) {
+      initializeMaxTsBlockLength(resultTsBlock);
+    }
+
+    if (resultTsBlock.getPositionCount() <= maxTupleSizeOfTsBlock) {
+      TsBlock res = resultTsBlock;
+      resultTsBlock = null;
+      return res;
+    } else {
+      retainedTsBlock = resultTsBlock;
+      resultTsBlock = null;
+      return getResultFromRetainedTsBlock();
+    }
+  }
+
+  public TsBlock getResultFromRetainedTsBlock() {
+    TsBlock res;
+    if (retainedTsBlock.getPositionCount() - startOffset <= maxTupleSizeOfTsBlock) {
+      res = retainedTsBlock.subTsBlock(startOffset);
+      retainedTsBlock = null;
+      startOffset = 0;
+    } else {
+      res = retainedTsBlock.getRegion(startOffset, maxTupleSizeOfTsBlock);
+      startOffset += maxTupleSizeOfTsBlock;
+    }
+    LOGGER.debug("Current tsBlock size is : {}", res.getRetainedSizeInBytes());
+    return res;
+  }
+
+  @Override
+  public OperatorContext getOperatorContext() {
+    return operatorContext;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/AbstractProcessOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/AbstractProcessOperator.java
new file mode 100644
index 0000000000..2e024256aa
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/AbstractProcessOperator.java
@@ -0,0 +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.mpp.execution.operator.process;
+
+import org.apache.iotdb.db.mpp.execution.operator.AbstractOperator;
+
+public abstract class AbstractProcessOperator extends AbstractOperator implements ProcessOperator {}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/RowBasedTimeJoinOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/RowBasedTimeJoinOperator.java
index 695893d400..56119832f5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/RowBasedTimeJoinOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/RowBasedTimeJoinOperator.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.db.mpp.execution.operator.process.join;
 
 import org.apache.iotdb.db.mpp.execution.operator.Operator;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
-import org.apache.iotdb.db.mpp.execution.operator.process.ProcessOperator;
+import org.apache.iotdb.db.mpp.execution.operator.process.AbstractProcessOperator;
 import org.apache.iotdb.db.mpp.execution.operator.process.join.merge.ColumnMerger;
 import org.apache.iotdb.db.mpp.execution.operator.process.join.merge.TimeComparator;
 import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
@@ -39,9 +39,7 @@ import java.util.List;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.util.concurrent.Futures.successfulAsList;
 
-public class RowBasedTimeJoinOperator implements ProcessOperator {
-
-  private final OperatorContext operatorContext;
+public class RowBasedTimeJoinOperator extends AbstractProcessOperator {
 
   private final List<Operator> children;
 
@@ -104,11 +102,11 @@ public class RowBasedTimeJoinOperator implements ProcessOperator {
     this.tsBlockBuilder = new TsBlockBuilder(dataTypes);
     this.mergers = mergers;
     this.comparator = comparator;
-  }
-
-  @Override
-  public OperatorContext getOperatorContext() {
-    return operatorContext;
+    this.maxReturnSize =
+        Math.min(
+            maxReturnSize,
+            (1L + outputColumnCount)
+                * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
   }
 
   @Override
@@ -127,6 +125,9 @@ public class RowBasedTimeJoinOperator implements ProcessOperator {
 
   @Override
   public TsBlock next() {
+    if (retainedTsBlock != null) {
+      return getResultFromRetainedTsBlock();
+    }
     tsBlockBuilder.reset();
     // end time for returned TsBlock this time, it's the min/max end time among all the children
     // TsBlocks order by asc/desc
@@ -198,7 +199,9 @@ public class RowBasedTimeJoinOperator implements ProcessOperator {
       }
       tsBlockBuilder.declarePosition();
     } while (currentTime < currentEndTime && !timeSelector.isEmpty());
-    return tsBlockBuilder.build();
+
+    resultTsBlock = tsBlockBuilder.build();
+    return checkTsBlockSizeAndGetResult();
   }
 
   @Override
@@ -206,6 +209,9 @@ public class RowBasedTimeJoinOperator implements ProcessOperator {
     if (finished) {
       return false;
     }
+    if (retainedTsBlock != null) {
+      return true;
+    }
     for (int i = 0; i < inputOperatorsCount; i++) {
       if (!empty(i)) {
         return true;
@@ -233,8 +239,11 @@ public class RowBasedTimeJoinOperator implements ProcessOperator {
     if (finished) {
       return true;
     }
-    finished = true;
+    if (retainedTsBlock != null) {
+      return false;
+    }
 
+    finished = true;
     for (int i = 0; i < inputOperatorsCount; i++) {
       // has more tsBlock output from children[i] or has cached tsBlock in inputTsBlocks[i]
       if (!noMoreTsBlocks[i] || !empty(i)) {
@@ -262,9 +271,7 @@ public class RowBasedTimeJoinOperator implements ProcessOperator {
 
   @Override
   public long calculateMaxReturnSize() {
-    // time + all value columns
-    return (1L + outputColumnCount)
-        * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
+    return maxReturnSize;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java
index 3170838986..cbb60d3c9b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/join/TimeJoinOperator.java
@@ -20,12 +20,11 @@ package org.apache.iotdb.db.mpp.execution.operator.process.join;
 
 import org.apache.iotdb.db.mpp.execution.operator.Operator;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
-import org.apache.iotdb.db.mpp.execution.operator.process.ProcessOperator;
+import org.apache.iotdb.db.mpp.execution.operator.process.AbstractProcessOperator;
 import org.apache.iotdb.db.mpp.execution.operator.process.join.merge.ColumnMerger;
 import org.apache.iotdb.db.mpp.execution.operator.process.join.merge.TimeComparator;
 import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.utils.datastructure.TimeSelector;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.block.TsBlock;
 import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
@@ -39,9 +38,8 @@ import java.util.List;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.util.concurrent.Futures.successfulAsList;
 
-public class TimeJoinOperator implements ProcessOperator {
-
-  private final OperatorContext operatorContext;
+@Deprecated
+public class TimeJoinOperator extends AbstractProcessOperator {
 
   private final List<Operator> children;
 
@@ -106,11 +104,6 @@ public class TimeJoinOperator implements ProcessOperator {
     this.comparator = comparator;
   }
 
-  @Override
-  public OperatorContext getOperatorContext() {
-    return operatorContext;
-  }
-
   @Override
   public ListenableFuture<?> isBlocked() {
     List<ListenableFuture<?>> listenableFutures = new ArrayList<>();
@@ -127,6 +120,9 @@ public class TimeJoinOperator implements ProcessOperator {
 
   @Override
   public TsBlock next() {
+    if (retainedTsBlock != null) {
+      return getResultFromRetainedTsBlock();
+    }
     tsBlockBuilder.reset();
     // end time for returned TsBlock this time, it's the min/max end time among all the children
     // TsBlocks order by asc/desc
@@ -197,7 +193,9 @@ public class TimeJoinOperator implements ProcessOperator {
     // update inputIndex using shadowInputIndex
     System.arraycopy(shadowInputIndex, 0, inputIndex, 0, inputOperatorsCount);
 
-    return tsBlockBuilder.build();
+    resultTsBlock = tsBlockBuilder.build();
+
+    return checkTsBlockSizeAndGetResult();
   }
 
   @Override
@@ -205,6 +203,9 @@ public class TimeJoinOperator implements ProcessOperator {
     if (finished) {
       return false;
     }
+    if (retainedTsBlock != null) {
+      return true;
+    }
     for (int i = 0; i < inputOperatorsCount; i++) {
       if (!empty(i)) {
         return true;
@@ -232,8 +233,11 @@ public class TimeJoinOperator implements ProcessOperator {
     if (finished) {
       return true;
     }
-    finished = true;
+    if (retainedTsBlock != null) {
+      return false;
+    }
 
+    finished = true;
     for (int i = 0; i < inputOperatorsCount; i++) {
       // has more tsBlock output from children[i] or has cached tsBlock in inputTsBlocks[i]
       if (!noMoreTsBlocks[i] || !empty(i)) {
@@ -262,8 +266,7 @@ public class TimeJoinOperator implements ProcessOperator {
   @Override
   public long calculateMaxReturnSize() {
     // time + all value columns
-    return (1L + outputColumnCount)
-        * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
+    return maxReturnSize;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractDataSourceOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractDataSourceOperator.java
new file mode 100644
index 0000000000..c810d61b64
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractDataSourceOperator.java
@@ -0,0 +1,32 @@
+/*
+ * 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.mpp.execution.operator.source;
+
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+
+public abstract class AbstractDataSourceOperator extends AbstractSourceOperator
+    implements DataSourceOperator {
+  protected SeriesScanUtil seriesScanUtil;
+
+  @Override
+  public void initQueryDataSource(QueryDataSource dataSource) {
+    seriesScanUtil.initQueryDataSource(dataSource);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSeriesAggregationScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSeriesAggregationScanOperator.java
index 15ad856245..5f285de538 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSeriesAggregationScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSeriesAggregationScanOperator.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.db.mpp.execution.operator.source;
 
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.mpp.aggregation.Aggregator;
 import org.apache.iotdb.db.mpp.aggregation.timerangeiterator.ITimeRangeIterator;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
@@ -43,14 +42,11 @@ import static org.apache.iotdb.db.mpp.execution.operator.AggregationUtil.appendA
 import static org.apache.iotdb.db.mpp.execution.operator.AggregationUtil.calculateAggregationFromRawData;
 import static org.apache.iotdb.db.mpp.execution.operator.AggregationUtil.isAllAggregatorsHasFinalResult;
 
-public abstract class AbstractSeriesAggregationScanOperator implements DataSourceOperator {
+public abstract class AbstractSeriesAggregationScanOperator extends AbstractDataSourceOperator {
 
-  protected final PlanNodeId sourceId;
-  protected final OperatorContext operatorContext;
   protected final boolean ascending;
   protected final boolean isGroupByQuery;
 
-  protected SeriesScanUtil seriesScanUtil;
   protected int subSensorSize;
 
   protected TsBlock inputTsBlock;
@@ -101,21 +97,6 @@ public abstract class AbstractSeriesAggregationScanOperator implements DataSourc
     this.maxReturnSize = maxReturnSize;
   }
 
-  @Override
-  public PlanNodeId getSourceId() {
-    return sourceId;
-  }
-
-  @Override
-  public OperatorContext getOperatorContext() {
-    return operatorContext;
-  }
-
-  @Override
-  public void initQueryDataSource(QueryDataSource dataSource) {
-    seriesScanUtil.initQueryDataSource(dataSource);
-  }
-
   @Override
   public long calculateMaxPeekMemory() {
     return maxRetainedSize + maxReturnSize;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSourceOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSourceOperator.java
new file mode 100644
index 0000000000..8a4bf66d36
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AbstractSourceOperator.java
@@ -0,0 +1,33 @@
+/*
+ * 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.mpp.execution.operator.source;
+
+import org.apache.iotdb.db.mpp.execution.operator.AbstractOperator;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+
+public abstract class AbstractSourceOperator extends AbstractOperator implements SourceOperator {
+
+  protected PlanNodeId sourceId;
+
+  @Override
+  public PlanNodeId getSourceId() {
+    return sourceId;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanOperator.java
index 9caaeeac60..388f5340f9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/AlignedSeriesScanOperator.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.db.mpp.execution.operator.source;
 
 import org.apache.iotdb.commons.path.AlignedPath;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -35,17 +34,12 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.concurrent.TimeUnit;
 
-public class AlignedSeriesScanOperator implements DataSourceOperator {
-
-  private final OperatorContext operatorContext;
-  private final AlignedSeriesScanUtil seriesScanUtil;
-  private final PlanNodeId sourceId;
+public class AlignedSeriesScanOperator extends AbstractDataSourceOperator {
 
   private final TsBlockBuilder builder;
+  private final int valueColumnCount;
   private boolean finished = false;
 
-  private final long maxReturnSize;
-
   public AlignedSeriesScanOperator(
       PlanNodeId sourceId,
       AlignedPath seriesPath,
@@ -64,26 +58,30 @@ public class AlignedSeriesScanOperator implements DataSourceOperator {
             valueFilter,
             ascending);
     // time + all value columns
-    this.maxReturnSize =
-        (1L + seriesPath.getMeasurementList().size())
-            * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
     this.builder = new TsBlockBuilder(seriesScanUtil.getTsDataTypeList());
-  }
-
-  @Override
-  public OperatorContext getOperatorContext() {
-    return operatorContext;
+    this.valueColumnCount = seriesPath.getColumnNum();
+    this.maxReturnSize =
+        Math.min(
+            maxReturnSize,
+            (1L + valueColumnCount)
+                * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
   }
 
   @Override
   public TsBlock next() {
-    TsBlock block = builder.build();
+    if (retainedTsBlock != null) {
+      return getResultFromRetainedTsBlock();
+    }
+    resultTsBlock = builder.build();
     builder.reset();
-    return block;
+    return checkTsBlockSizeAndGetResult();
   }
 
   @Override
   public boolean hasNext() {
+    if (retainedTsBlock != null) {
+      return true;
+    }
     try {
 
       // start stopwatch
@@ -131,7 +129,9 @@ public class AlignedSeriesScanOperator implements DataSourceOperator {
 
   @Override
   public long calculateMaxPeekMemory() {
-    return maxReturnSize;
+    return Math.max(
+        maxReturnSize,
+        (1L + valueColumnCount) * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
   }
 
   @Override
@@ -141,7 +141,7 @@ public class AlignedSeriesScanOperator implements DataSourceOperator {
 
   @Override
   public long calculateRetainedSizeAfterCallingNext() {
-    return 0L;
+    return calculateMaxPeekMemory() - calculateMaxReturnSize();
   }
 
   private boolean readFileData() throws IOException {
@@ -205,14 +205,4 @@ public class AlignedSeriesScanOperator implements DataSourceOperator {
   private boolean isEmpty(TsBlock tsBlock) {
     return tsBlock == null || tsBlock.isEmpty();
   }
-
-  @Override
-  public PlanNodeId getSourceId() {
-    return sourceId;
-  }
-
-  @Override
-  public void initQueryDataSource(QueryDataSource dataSource) {
-    seriesScanUtil.initQueryDataSource(dataSource);
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java
index fd831fcd6a..681ffb2628 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/source/SeriesScanOperator.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.db.mpp.execution.operator.source;
 
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.mpp.execution.operator.OperatorContext;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -36,17 +35,11 @@ import java.io.IOException;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-public class SeriesScanOperator implements DataSourceOperator {
+public class SeriesScanOperator extends AbstractDataSourceOperator {
 
-  private final OperatorContext operatorContext;
-  private final SeriesScanUtil seriesScanUtil;
-  private final PlanNodeId sourceId;
   private final TsBlockBuilder builder;
-
   private boolean finished = false;
 
-  private final long maxReturnSize;
-
   public SeriesScanOperator(
       PlanNodeId sourceId,
       PartialPath seriesPath,
@@ -67,24 +60,26 @@ public class SeriesScanOperator implements DataSourceOperator {
             timeFilter,
             valueFilter,
             ascending);
-    this.maxReturnSize = TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
+    this.maxReturnSize =
+        Math.min(maxReturnSize, TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
     this.builder = new TsBlockBuilder(seriesScanUtil.getTsDataTypeList());
   }
 
-  @Override
-  public OperatorContext getOperatorContext() {
-    return operatorContext;
-  }
-
   @Override
   public TsBlock next() {
-    TsBlock block = builder.build();
+    if (retainedTsBlock != null) {
+      return getResultFromRetainedTsBlock();
+    }
+    resultTsBlock = builder.build();
     builder.reset();
-    return block;
+    return checkTsBlockSizeAndGetResult();
   }
 
   @Override
   public boolean hasNext() {
+    if (retainedTsBlock != null) {
+      return true;
+    }
     try {
 
       // start stopwatch
@@ -132,7 +127,7 @@ public class SeriesScanOperator implements DataSourceOperator {
 
   @Override
   public long calculateMaxPeekMemory() {
-    return maxReturnSize;
+    return Math.max(maxReturnSize, TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
   }
 
   @Override
@@ -142,7 +137,7 @@ public class SeriesScanOperator implements DataSourceOperator {
 
   @Override
   public long calculateRetainedSizeAfterCallingNext() {
-    return 0L;
+    return calculateMaxPeekMemory() - calculateMaxReturnSize();
   }
 
   private boolean readFileData() throws IOException {
@@ -203,14 +198,4 @@ public class SeriesScanOperator implements DataSourceOperator {
   private boolean isEmpty(TsBlock tsBlock) {
     return tsBlock == null || tsBlock.isEmpty();
   }
-
-  @Override
-  public PlanNodeId getSourceId() {
-    return sourceId;
-  }
-
-  @Override
-  public void initQueryDataSource(QueryDataSource dataSource) {
-    seriesScanUtil.initQueryDataSource(dataSource);
-  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
index 585ed15f43..2652173e47 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/execution/operator/OperatorMemoryTest.java
@@ -195,14 +195,20 @@ public class OperatorMemoryTest {
               null,
               true);
 
-      assertEquals(
-          4 * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte(),
-          seriesScanOperator.calculateMaxPeekMemory());
-      assertEquals(
-          4 * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte(),
-          seriesScanOperator.calculateMaxReturnSize());
+      long maxPeekMemory =
+          Math.max(
+              TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(),
+              4 * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
+      long maxReturnMemory =
+          Math.min(
+              TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(),
+              4 * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
+      assertEquals(maxPeekMemory, seriesScanOperator.calculateMaxPeekMemory());
+      assertEquals(maxReturnMemory, seriesScanOperator.calculateMaxReturnSize());
 
-      assertEquals(0, seriesScanOperator.calculateRetainedSizeAfterCallingNext());
+      assertEquals(
+          maxPeekMemory - maxReturnMemory,
+          seriesScanOperator.calculateRetainedSizeAfterCallingNext());
 
     } catch (IllegalPathException e) {
       e.printStackTrace();
@@ -416,7 +422,9 @@ public class OperatorMemoryTest {
     dataTypeList.add(TSDataType.INT32);
     dataTypeList.add(TSDataType.INT32);
     long expectedMaxReturnSize =
-        3L * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
+        Math.min(
+            TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(),
+            3L * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
     long expectedMaxPeekMemory = 0;
     long childrenMaxPeekMemory = 0;
 
@@ -458,7 +466,9 @@ public class OperatorMemoryTest {
     dataTypeList.add(TSDataType.INT32);
     dataTypeList.add(TSDataType.INT32);
     long expectedMaxReturnSize =
-        3L * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte();
+        Math.min(
+            TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(),
+            3L * TSFileDescriptor.getInstance().getConfig().getPageSizeInByte());
     long expectedMaxPeekMemory = 0;
     long childrenMaxPeekMemory = 0;
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlock.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlock.java
index 3c2f6a58d6..9f37f4b780 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlock.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlock.java
@@ -479,6 +479,15 @@ public class TsBlock {
     return retainedSizeInBytes;
   }
 
+  public int getTotalInstanceSize() {
+    int totalInstanceSize = INSTANCE_SIZE;
+    totalInstanceSize += timeColumn.getInstanceSize();
+    for (Column column : valueColumns) {
+      totalInstanceSize += column.getInstanceSize();
+    }
+    return totalInstanceSize;
+  }
+
   private static int determinePositionCount(Column... columns) {
     requireNonNull(columns, "columns is null");
     if (columns.length == 0) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BinaryColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BinaryColumn.java
index aeca614dfe..0aa80cff18 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BinaryColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BinaryColumn.java
@@ -162,4 +162,9 @@ public class BinaryColumn implements Column {
       }
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BooleanColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BooleanColumn.java
index 9da7b58b04..39995db4e1 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BooleanColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/BooleanColumn.java
@@ -160,4 +160,9 @@ public class BooleanColumn implements Column {
       }
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/Column.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/Column.java
index fe2af36fca..203ea58b25 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/Column.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/Column.java
@@ -149,4 +149,6 @@ public interface Column {
 
   /** reverse the column */
   void reverse();
+
+  int getInstanceSize();
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/DoubleColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/DoubleColumn.java
index 56f0cc7de2..c7966f1653 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/DoubleColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/DoubleColumn.java
@@ -161,4 +161,9 @@ public class DoubleColumn implements Column {
       }
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/FloatColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/FloatColumn.java
index bb3e180ab1..cd96c17812 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/FloatColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/FloatColumn.java
@@ -159,4 +159,9 @@ public class FloatColumn implements Column {
       }
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/IntColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/IntColumn.java
index 38cf078a9f..b0ac59aef5 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/IntColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/IntColumn.java
@@ -159,4 +159,9 @@ public class IntColumn implements Column {
       }
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/LongColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/LongColumn.java
index a67fef0fa1..cf7e64da5d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/LongColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/LongColumn.java
@@ -159,4 +159,9 @@ public class LongColumn implements Column {
       }
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/NullColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/NullColumn.java
index d9307d44c0..b50bc77a2f 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/NullColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/NullColumn.java
@@ -117,4 +117,9 @@ public class NullColumn implements Column {
         throw new IllegalArgumentException("Unknown data type: " + dataType);
     }
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/RunLengthEncodedColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/RunLengthEncodedColumn.java
index b512b9f252..54174d4776 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/RunLengthEncodedColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/RunLengthEncodedColumn.java
@@ -208,4 +208,9 @@ public class RunLengthEncodedColumn implements Column {
   public void reverse() {
     // do nothing because the underlying column has only one value
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/TimeColumn.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/TimeColumn.java
index 4245bff643..ee6e6d3699 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/TimeColumn.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/column/TimeColumn.java
@@ -142,4 +142,9 @@ public class TimeColumn implements Column {
   public long[] getTimes() {
     return values;
   }
+
+  @Override
+  public int getInstanceSize() {
+    return INSTANCE_SIZE;
+  }
 }