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

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

commit 8e5b9b7463b4582604add685acd823fbf13789a9
Author: JackieTien97 <[email protected]>
AuthorDate: Tue Nov 12 18:34:55 2024 +0800

    Add cache hit situation and actual io size for BloomFilter, 
TimeSeriesMetadata(including ChunkMetadatList) and Chunk
---
 .../execution/fragment/QueryStatistics.java        |  53 +++++++++
 .../execution/operator/source/FileLoaderUtils.java |   9 +-
 .../db/storageengine/buffer/BloomFilterCache.java  |  80 +++++++++----
 .../iotdb/db/storageengine/buffer/ChunkCache.java  | 125 ++++++++++++++++-----
 .../buffer/TimeSeriesMetadataCache.java            |  65 +++++++++--
 .../dataregion/read/control/FileReaderManager.java |  32 ++++--
 .../read/reader/chunk/DiskAlignedChunkLoader.java  |   6 +-
 .../read/reader/chunk/DiskChunkLoader.java         |   7 +-
 .../db/storageengine/buffer/ChunkCacheTest.java    |   6 +-
 .../commons/exception/IoTDBIORuntimeException.java |  14 +++
 pom.xml                                            |   2 +-
 11 files changed, 316 insertions(+), 83 deletions(-)

diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java
index b996569a69d..179cc446e00 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryStatistics.java
@@ -29,6 +29,11 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public class QueryStatistics {
 
+  private final AtomicLong loadBloomFilterFromCacheCount = new AtomicLong(0);
+  private final AtomicLong loadBloomFilterFromDiskCount = new AtomicLong(0);
+  private final AtomicLong loadBloomFilterActualIOSize = new AtomicLong(0);
+  private final AtomicLong loadBloomFilterTime = new AtomicLong(0);
+
   // statistics for count and time of load timeseriesmetadata
   private final AtomicLong loadTimeSeriesMetadataDiskSeqCount = new 
AtomicLong(0);
   private final AtomicLong loadTimeSeriesMetadataDiskUnSeqCount = new 
AtomicLong(0);
@@ -48,6 +53,10 @@ public class QueryStatistics {
   private final AtomicLong loadTimeSeriesMetadataAlignedMemSeqTime = new 
AtomicLong(0);
   private final AtomicLong loadTimeSeriesMetadataAlignedMemUnSeqTime = new 
AtomicLong(0);
 
+  private final AtomicLong loadTimeSeriesMetadataFromCacheCount = new 
AtomicLong(0);
+  private final AtomicLong loadTimeSeriesMetadataFromDiskCount = new 
AtomicLong(0);
+  private final AtomicLong loadTimeSeriesMetadataActualIOSize = new 
AtomicLong(0);
+
   // statistics for count and time of construct chunk readers(disk io and 
decompress)
   private final AtomicLong constructNonAlignedChunkReadersDiskCount = new 
AtomicLong(0);
   private final AtomicLong constructNonAlignedChunkReadersMemCount = new 
AtomicLong(0);
@@ -59,6 +68,10 @@ public class QueryStatistics {
   private final AtomicLong constructAlignedChunkReadersDiskTime = new 
AtomicLong(0);
   private final AtomicLong constructAlignedChunkReadersMemTime = new 
AtomicLong(0);
 
+  private final AtomicLong loadChunkFromCacheCount = new AtomicLong(0);
+  private final AtomicLong loadChunkFromDiskCount = new AtomicLong(0);
+  private final AtomicLong loadChunkActualIOSize = new AtomicLong(0);
+
   // statistics for count and time of page decode
   private final AtomicLong pageReadersDecodeAlignedDiskCount = new 
AtomicLong(0);
   private final AtomicLong pageReadersDecodeAlignedDiskTime = new 
AtomicLong(0);
@@ -225,6 +238,46 @@ public class QueryStatistics {
     return pageReaderMaxUsedMemorySize;
   }
 
+  public AtomicLong getLoadBloomFilterActualIOSize() {
+    return loadBloomFilterActualIOSize;
+  }
+
+  public AtomicLong getLoadBloomFilterFromCacheCount() {
+    return loadBloomFilterFromCacheCount;
+  }
+
+  public AtomicLong getLoadBloomFilterFromDiskCount() {
+    return loadBloomFilterFromDiskCount;
+  }
+
+  public AtomicLong getLoadBloomFilterTime() {
+    return loadBloomFilterTime;
+  }
+
+  public AtomicLong getLoadChunkActualIOSize() {
+    return loadChunkActualIOSize;
+  }
+
+  public AtomicLong getLoadChunkFromCacheCount() {
+    return loadChunkFromCacheCount;
+  }
+
+  public AtomicLong getLoadChunkFromDiskCount() {
+    return loadChunkFromDiskCount;
+  }
+
+  public AtomicLong getLoadTimeSeriesMetadataActualIOSize() {
+    return loadTimeSeriesMetadataActualIOSize;
+  }
+
+  public AtomicLong getLoadTimeSeriesMetadataFromCacheCount() {
+    return loadTimeSeriesMetadataFromCacheCount;
+  }
+
+  public AtomicLong getLoadTimeSeriesMetadataFromDiskCount() {
+    return loadTimeSeriesMetadataFromDiskCount;
+  }
+
   public TQueryStatistics toThrift() {
     return new TQueryStatistics(
         loadTimeSeriesMetadataDiskSeqCount.get(),
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
index 1941f3c739f..e940922a96a 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/FileLoaderUtils.java
@@ -102,7 +102,8 @@ public class FileLoaderUtils {
                         seriesPath.getMeasurement()),
                     allSensors,
                     resource.getTimeIndexType() == 
ITimeIndex.FILE_TIME_INDEX_TYPE,
-                    context.isDebug());
+                    context.isDebug(),
+                    context);
         if (timeSeriesMetadata != null) {
           long t2 = System.nanoTime();
           List<Modification> pathModifications =
@@ -276,7 +277,8 @@ public class FileLoaderUtils {
             new TimeSeriesMetadataCacheKey(resource.getTsFileID(), deviceId, 
""),
             allSensors,
             resource.getTimeIndexType() == ITimeIndex.FILE_TIME_INDEX_TYPE,
-            isDebug);
+            isDebug,
+            context);
     if (timeColumn != null) {
       // only need time column, like count_time aggregation
       if (valueMeasurementList.isEmpty()) {
@@ -304,7 +306,8 @@ public class FileLoaderUtils {
                       resource.getTsFileID(), deviceId, valueMeasurement),
                   allSensors,
                   resource.getTimeIndexType() == 
ITimeIndex.FILE_TIME_INDEX_TYPE,
-                  isDebug);
+                  isDebug,
+                  context);
           exist = (exist || (valueColumn != null));
           valueTimeSeriesMetadataList.add(valueColumn);
         }
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java
index ca2f3479908..03c56bc4223 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/BloomFilterCache.java
@@ -19,13 +19,14 @@
 
 package org.apache.iotdb.db.storageengine.buffer;
 
+import org.apache.iotdb.commons.exception.IoTDBIORuntimeException;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import 
org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager;
 
+import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
-import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.github.benmanes.caffeine.cache.Weigher;
 import org.apache.tsfile.read.TsFileSequenceReader;
 import org.apache.tsfile.utils.BloomFilter;
@@ -36,6 +37,8 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.Objects;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.function.LongConsumer;
 
 /** This class is used to cache <code>BloomFilter</code> in IoTDB. The caching 
strategy is LRU. */
 @SuppressWarnings("squid:S6548")
@@ -49,7 +52,7 @@ public class BloomFilterCache {
   private static final boolean CACHE_ENABLE = CONFIG.isMetaDataCacheEnable();
   private final AtomicLong entryAverageSize = new AtomicLong(0);
 
-  private final LoadingCache<BloomFilterCacheKey, BloomFilter> lruCache;
+  private final Cache<BloomFilterCacheKey, BloomFilter> lruCache;
 
   private BloomFilterCache() {
     if (CACHE_ENABLE) {
@@ -63,35 +66,48 @@ public class BloomFilterCache {
                     (key, bloomFilter) ->
                         (int) (key.getRetainedSizeInBytes() + 
bloomFilter.getRetainedSizeInBytes()))
             .recordStats()
-            .build(
-                key -> {
-                  TsFileSequenceReader reader =
-                      FileReaderManager.getInstance().get(key.filePath, true);
-                  return reader.readBloomFilter();
-                });
+            .build();
   }
 
   public static BloomFilterCache getInstance() {
     return BloomFilterCacheHolder.INSTANCE;
   }
 
+  @TestOnly
   public BloomFilter get(BloomFilterCacheKey key) throws IOException {
-    return get(key, false);
+    LongConsumer emptyConsumer = l -> {};
+    return get(key, false, emptyConsumer, emptyConsumer, emptyConsumer);
   }
 
-  public BloomFilter get(BloomFilterCacheKey key, boolean debug) throws 
IOException {
-    if (!CACHE_ENABLE) {
-      TsFileSequenceReader reader = 
FileReaderManager.getInstance().get(key.filePath, true);
-      return reader.readBloomFilter();
-    }
+  public BloomFilter get(
+      BloomFilterCacheKey key,
+      boolean debug,
+      LongConsumer ioSizeRecorder,
+      LongConsumer cacheHitAdder,
+      LongConsumer cacheMissAdder)
+      throws IOException {
+    BloomFilterLoader loader = new BloomFilterLoader(ioSizeRecorder);
+    try {
+      if (!CACHE_ENABLE) {
+        return loader.apply(key);
+      }
 
-    BloomFilter bloomFilter = lruCache.get(key);
+      BloomFilter bloomFilter = lruCache.get(key, loader);
 
-    if (debug) {
-      DEBUG_LOGGER.info("get bloomFilter from cache where filePath is: {}", 
key.filePath);
-    }
+      if (debug) {
+        DEBUG_LOGGER.info("get bloomFilter from cache where filePath is: {}", 
key.filePath);
+      }
 
-    return bloomFilter;
+      return bloomFilter;
+    } catch (IoTDBIORuntimeException e) {
+      throw e.getCause();
+    } finally {
+      if (loader.cacheMiss) {
+        cacheMissAdder.accept(1);
+      } else {
+        cacheHitAdder.accept(1);
+      }
+    }
   }
 
   public double calculateBloomFilterHitRatio() {
@@ -183,6 +199,32 @@ public class BloomFilterCache {
     }
   }
 
+  private static class BloomFilterLoader implements 
Function<BloomFilterCacheKey, BloomFilter> {
+
+    private boolean cacheMiss = false;
+    private final LongConsumer ioSizeRecorder;
+
+    private BloomFilterLoader(LongConsumer ioSizeRecorder) {
+      this.ioSizeRecorder = ioSizeRecorder;
+    }
+
+    @Override
+    public BloomFilter apply(BloomFilterCacheKey bloomFilterCacheKey) {
+      try {
+        cacheMiss = true;
+        TsFileSequenceReader reader =
+            FileReaderManager.getInstance().get(bloomFilterCacheKey.filePath, 
true, ioSizeRecorder);
+        return reader.readBloomFilter(ioSizeRecorder);
+      } catch (IOException e) {
+        throw new IoTDBIORuntimeException(e);
+      }
+    }
+
+    public boolean isCacheMiss() {
+      return cacheMiss;
+    }
+  }
+
   /** singleton pattern. */
   private static class BloomFilterCacheHolder {
     private static final BloomFilterCache INSTANCE = new BloomFilterCache();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java
index 3ad81d39930..719a550d386 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/ChunkCache.java
@@ -19,17 +19,19 @@
 
 package org.apache.iotdb.db.storageengine.buffer;
 
+import org.apache.iotdb.commons.exception.IoTDBIORuntimeException;
 import org.apache.iotdb.commons.service.metric.MetricService;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext;
 import org.apache.iotdb.db.queryengine.metric.ChunkCacheMetrics;
 import org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet;
 import 
org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager;
 import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID;
 
+import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
-import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.github.benmanes.caffeine.cache.Weigher;
 import org.apache.tsfile.file.metadata.statistics.Statistics;
 import org.apache.tsfile.read.TsFileSequenceReader;
@@ -42,6 +44,8 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.List;
 import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.LongConsumer;
 
 import static 
org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet.READ_CHUNK_ALL;
 import static 
org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet.READ_CHUNK_FILE;
@@ -64,7 +68,7 @@ public class ChunkCache {
       SeriesScanCostMetricSet.getInstance();
 
   // to save memory footprint, we don't save measurementId in ChunkHeader of 
Chunk
-  private final LoadingCache<ChunkCacheKey, Chunk> lruCache;
+  private final Cache<ChunkCacheKey, Chunk> lruCache;
 
   private ChunkCache() {
     if (CACHE_ENABLE) {
@@ -78,21 +82,7 @@ public class ChunkCache {
                     (key, chunk) ->
                         (int) (key.getRetainedSizeInBytes() + 
chunk.getRetainedSizeInBytes()))
             .recordStats()
-            .build(
-                key -> {
-                  long startTime = System.nanoTime();
-                  try {
-                    TsFileSequenceReader reader =
-                        FileReaderManager.getInstance().get(key.getFilePath(), 
key.closed);
-                    Chunk chunk = reader.readMemChunk(key.offsetOfChunkHeader);
-                    // to save memory footprint, we don't save measurementId 
in ChunkHeader of Chunk
-                    chunk.getHeader().setMeasurementID(null);
-                    return chunk;
-                  } finally {
-                    SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost(
-                        READ_CHUNK_FILE, System.nanoTime() - startTime);
-                  }
-                });
+            .build();
 
     // add metrics
     MetricService.getInstance().addMetricSet(new ChunkCacheMetrics(this));
@@ -106,36 +96,85 @@ public class ChunkCache {
     return ChunkCacheHolder.INSTANCE;
   }
 
+  @TestOnly
+  public Chunk get(
+      ChunkCacheKey chunkCacheKey, List<TimeRange> timeRangeList, Statistics 
chunkStatistic)
+      throws IOException {
+    LongConsumer emptyConsumer = l -> {};
+    return get(
+        chunkCacheKey,
+        timeRangeList,
+        chunkStatistic,
+        false,
+        emptyConsumer,
+        emptyConsumer,
+        emptyConsumer);
+  }
+
   public Chunk get(
       ChunkCacheKey chunkCacheKey,
       List<TimeRange> timeRangeList,
       Statistics chunkStatistic,
-      boolean debug)
+      QueryContext queryContext)
+      throws IOException {
+    LongConsumer ioSizeRecorder =
+        
queryContext.getQueryStatistics().getLoadChunkActualIOSize()::addAndGet;
+    LongConsumer cacheHitAdder =
+        
queryContext.getQueryStatistics().getLoadChunkFromCacheCount()::addAndGet;
+    LongConsumer cacheMissAdder =
+        
queryContext.getQueryStatistics().getLoadChunkFromDiskCount()::addAndGet;
+    return get(
+        chunkCacheKey,
+        timeRangeList,
+        chunkStatistic,
+        queryContext.isDebug(),
+        ioSizeRecorder,
+        cacheHitAdder,
+        cacheMissAdder);
+  }
+
+  private Chunk get(
+      ChunkCacheKey chunkCacheKey,
+      List<TimeRange> timeRangeList,
+      Statistics chunkStatistic,
+      boolean debug,
+      LongConsumer ioSizeRecorder,
+      LongConsumer cacheHitAdder,
+      LongConsumer cacheMissAdder)
       throws IOException {
     long startTime = System.nanoTime();
+    ChunkLoader chunkLoader = new ChunkLoader(ioSizeRecorder);
     try {
       if (!CACHE_ENABLE) {
-        TsFileSequenceReader reader =
-            FileReaderManager.getInstance().get(chunkCacheKey.getFilePath(), 
true);
-        Chunk chunk = reader.readMemChunk(chunkCacheKey.offsetOfChunkHeader);
-        return new Chunk(
-            chunk.getHeader(), chunk.getData().duplicate(), timeRangeList, 
chunkStatistic);
+        Chunk chunk = chunkLoader.apply(chunkCacheKey);
+        return constructChunk(chunk, timeRangeList, chunkStatistic);
       }
 
-      Chunk chunk = lruCache.get(chunkCacheKey);
+      Chunk chunk = lruCache.get(chunkCacheKey, chunkLoader);
 
       if (debug) {
         DEBUG_LOGGER.info("get chunk from cache whose key is: {}", 
chunkCacheKey);
       }
 
-      return new Chunk(
-          chunk.getHeader(), chunk.getData().duplicate(), timeRangeList, 
chunkStatistic);
+      return constructChunk(chunk, timeRangeList, chunkStatistic);
+    } catch (IoTDBIORuntimeException e) {
+      throw e.getCause();
     } finally {
+      if (chunkLoader.cacheMiss) {
+        cacheMissAdder.accept(1);
+      } else {
+        cacheHitAdder.accept(1);
+      }
       SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost(
           READ_CHUNK_ALL, System.nanoTime() - startTime);
     }
   }
 
+  private Chunk constructChunk(
+      Chunk chunk, List<TimeRange> timeRangeList, Statistics chunkStatistic) {
+    return new Chunk(chunk.getHeader(), chunk.getData().duplicate(), 
timeRangeList, chunkStatistic);
+  }
+
   public double calculateChunkHitRatio() {
     return lruCache.stats().hitRate();
   }
@@ -245,6 +284,40 @@ public class ChunkCache {
     }
   }
 
+  private static class ChunkLoader implements Function<ChunkCacheKey, Chunk> {
+
+    private boolean cacheMiss = false;
+    private final LongConsumer ioSizeRecorder;
+
+    private ChunkLoader(LongConsumer ioSizeRecorder) {
+      this.ioSizeRecorder = ioSizeRecorder;
+    }
+
+    @Override
+    public Chunk apply(ChunkCacheKey key) {
+
+      long startTime = System.nanoTime();
+      try {
+
+        TsFileSequenceReader reader =
+            FileReaderManager.getInstance().get(key.getFilePath(), key.closed);
+        Chunk chunk = reader.readMemChunk(key.offsetOfChunkHeader);
+        // to save memory footprint, we don't save measurementId in 
ChunkHeader of Chunk
+        chunk.getHeader().setMeasurementID(null);
+        return chunk;
+      } catch (IOException e) {
+        throw new IoTDBIORuntimeException(e);
+      } finally {
+        SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost(
+            READ_CHUNK_FILE, System.nanoTime() - startTime);
+      }
+    }
+
+    public boolean isCacheMiss() {
+      return cacheMiss;
+    }
+  }
+
   /** singleton pattern. */
   private static class ChunkCacheHolder {
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java
index 2e7e2b2b627..dfa2e104f04 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/buffer/TimeSeriesMetadataCache.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.commons.service.metric.MetricService;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext;
 import org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet;
 import org.apache.iotdb.db.queryengine.metric.TimeSeriesMetadataCacheMetrics;
 import 
org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager;
@@ -50,6 +51,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.WeakHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
 
 import static 
org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet.READ_TIMESERIES_METADATA_CACHE;
 import static 
org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet.READ_TIMESERIES_METADATA_FILE;
@@ -108,9 +110,15 @@ public class TimeSeriesMetadataCache {
       TimeSeriesMetadataCacheKey key,
       Set<String> allSensors,
       boolean ignoreNotExists,
-      boolean debug)
+      boolean debug,
+      QueryContext queryContext)
       throws IOException {
     long startTime = System.nanoTime();
+    long loadBloomFilterTime = 0;
+    LongConsumer timeSeriesMetadataIoSizeRecorder =
+        
queryContext.getQueryStatistics().getLoadTimeSeriesMetadataActualIOSize()::addAndGet;
+    LongConsumer bloomFilterIoSizeRecorder =
+        
queryContext.getQueryStatistics().getLoadBloomFilterActualIOSize()::addAndGet;
     boolean cacheHit = true;
     try {
       String deviceStringFormat = key.device.toString();
@@ -118,15 +126,21 @@ public class TimeSeriesMetadataCache {
         cacheHit = false;
 
         // bloom filter part
-        TsFileSequenceReader reader = 
FileReaderManager.getInstance().get(filePath, true);
-        BloomFilter bloomFilter = reader.readBloomFilter();
+        TsFileSequenceReader reader =
+            FileReaderManager.getInstance().get(filePath, true, 
bloomFilterIoSizeRecorder);
+        BloomFilter bloomFilter = 
reader.readBloomFilter(bloomFilterIoSizeRecorder);
+        
queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount().incrementAndGet();
         if (bloomFilter != null
             && !bloomFilter.contains(
                 deviceStringFormat + IoTDBConstant.PATH_SEPARATOR + 
key.measurement)) {
+          loadBloomFilterTime = System.nanoTime() - startTime;
           return null;
         }
+        loadBloomFilterTime = System.nanoTime() - startTime;
+
         TimeseriesMetadata timeseriesMetadata =
-            reader.readTimeseriesMetadata(key.device, key.measurement, 
ignoreNotExists);
+            reader.readTimeseriesMetadata(
+                key.device, key.measurement, ignoreNotExists, 
timeSeriesMetadataIoSizeRecorder);
         return (timeseriesMetadata == null || 
timeseriesMetadata.getStatistics().getCount() == 0)
             ? null
             : timeseriesMetadata;
@@ -148,6 +162,7 @@ public class TimeSeriesMetadataCache {
           if (timeseriesMetadata == null) {
             cacheHit = false;
 
+            long loadBloomFilterStartTime = System.nanoTime();
             // bloom filter part
             BloomFilter bloomFilter =
                 BloomFilterCache.getInstance()
@@ -158,18 +173,33 @@ public class TimeSeriesMetadataCache {
                             key.timePartitionId,
                             key.tsFileVersion,
                             key.compactionVersion),
-                        debug);
+                        debug,
+                        bloomFilterIoSizeRecorder,
+                        
queryContext.getQueryStatistics().getLoadBloomFilterFromCacheCount()
+                            ::addAndGet,
+                        
queryContext.getQueryStatistics().getLoadBloomFilterFromDiskCount()
+                            ::addAndGet);
             if (bloomFilter != null
                 && !bloomFilter.contains(
                     deviceStringFormat + TsFileConstant.PATH_SEPARATOR + 
key.measurement)) {
               if (debug) {
                 DEBUG_LOGGER.info("TimeSeries meta data {} is filter by 
bloomFilter!", key);
               }
+              loadBloomFilterTime = System.nanoTime() - 
loadBloomFilterStartTime;
               return null;
             }
-            TsFileSequenceReader reader = 
FileReaderManager.getInstance().get(filePath, true);
+
+            loadBloomFilterTime = System.nanoTime() - loadBloomFilterStartTime;
+            TsFileSequenceReader reader =
+                FileReaderManager.getInstance()
+                    .get(filePath, true, timeSeriesMetadataIoSizeRecorder);
             List<TimeseriesMetadata> timeSeriesMetadataList =
-                reader.readTimeseriesMetadata(key.device, key.measurement, 
allSensors);
+                reader.readTimeseriesMetadata(
+                    key.device,
+                    key.measurement,
+                    allSensors,
+                    ignoreNotExists,
+                    timeSeriesMetadataIoSizeRecorder);
             // put TimeSeriesMetadata of all sensors used in this read into 
cache
             for (TimeseriesMetadata metadata : timeSeriesMetadataList) {
               TimeSeriesMetadataCacheKey k =
@@ -207,9 +237,24 @@ public class TimeSeriesMetadataCache {
         return new TimeseriesMetadata(timeseriesMetadata);
       }
     } finally {
-      SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost(
-          cacheHit ? READ_TIMESERIES_METADATA_CACHE : 
READ_TIMESERIES_METADATA_FILE,
-          System.nanoTime() - startTime);
+      
queryContext.getQueryStatistics().getLoadBloomFilterTime().getAndAdd(loadBloomFilterTime);
+      if (cacheHit) {
+        queryContext
+            .getQueryStatistics()
+            .getLoadTimeSeriesMetadataFromCacheCount()
+            .incrementAndGet();
+        // in metric panel, loading BloomFilter time is included in loading 
TimeSeriesMetadata
+        SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost(
+            READ_TIMESERIES_METADATA_CACHE, System.nanoTime() - startTime);
+      } else {
+        queryContext
+            .getQueryStatistics()
+            .getLoadTimeSeriesMetadataFromDiskCount()
+            .incrementAndGet();
+        // in metric panel, loading BloomFilter time is included in loading 
TimeSeriesMetadata
+        SERIES_SCAN_COST_METRIC_SET.recordSeriesScanCost(
+            READ_TIMESERIES_METADATA_FILE, System.nanoTime() - startTime);
+      }
     }
   }
 
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java
index b3189b1ff52..29c4e79550a 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/FileReaderManager.java
@@ -22,7 +22,6 @@ package 
org.apache.iotdb.db.storageengine.dataregion.read.control;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
 
-import org.apache.tsfile.common.conf.TSFileConfig;
 import org.apache.tsfile.read.TsFileSequenceReader;
 import org.apache.tsfile.read.UnClosedTsFileReader;
 import org.slf4j.Logger;
@@ -33,6 +32,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.LongConsumer;
 
 /**
  * {@link FileReaderManager} is a singleton, which is used to manage all file 
readers(opened file
@@ -114,6 +114,23 @@ public class FileReaderManager {
   @SuppressWarnings("squid:S2095")
   public synchronized TsFileSequenceReader get(String filePath, boolean 
isClosed)
       throws IOException {
+    return get(filePath, isClosed, null);
+  }
+
+  /**
+   * Get the reader of the file(tsfile or unseq tsfile) indicated by filePath. 
If the reader already
+   * exists, just get it from closedFileReaderMap or unclosedFileReaderMap 
depending on isClosing .
+   * Otherwise a new reader will be created and cached.
+   *
+   * @param filePath the path of the file, of which the reader is desired.
+   * @param isClosed whether the corresponding file still receives insertions 
or not.
+   * @param ioSizeRecorder can be null
+   * @return the reader of the file specified by filePath.
+   * @throws IOException when reader cannot be created.
+   */
+  @SuppressWarnings("squid:S2095")
+  public synchronized TsFileSequenceReader get(
+      String filePath, boolean isClosed, LongConsumer ioSizeRecorder) throws 
IOException {
 
     Map<String, TsFileSequenceReader> readerMap =
         !isClosed ? unclosedFileReaderMap : closedFileReaderMap;
@@ -127,17 +144,10 @@ public class FileReaderManager {
       TsFileSequenceReader tsFileReader = null;
       // check if the file is old version
       if (!isClosed) {
-        tsFileReader = new UnClosedTsFileReader(filePath);
+        tsFileReader = new UnClosedTsFileReader(filePath, ioSizeRecorder);
       } else {
-        tsFileReader = new TsFileSequenceReader(filePath);
-        byte versionNumber = tsFileReader.readVersionNumber();
-        if (versionNumber != TSFileConfig.VERSION_NUMBER
-            && versionNumber != TSFileConfig.VERSION_NUMBER_V3) {
-          tsFileReader.close();
-          throw new IOException(
-              String.format(
-                  "The version of TsFile %s is not correct: %s", filePath, 
versionNumber));
-        }
+        // already do the version check in TsFileSequenceReader's constructor
+        tsFileReader = new TsFileSequenceReader(filePath, ioSizeRecorder);
       }
       readerMap.put(filePath, tsFileReader);
       return tsFileReader;
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java
index d2f2a2da02f..adef246ec8d 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskAlignedChunkLoader.java
@@ -44,7 +44,6 @@ import static 
org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet.INI
 public class DiskAlignedChunkLoader implements IChunkLoader {
 
   private final QueryContext context;
-  private final boolean debug;
 
   private final TsFileResource resource;
 
@@ -58,7 +57,6 @@ public class DiskAlignedChunkLoader implements IChunkLoader {
   public DiskAlignedChunkLoader(
       QueryContext context, TsFileResource resource, boolean 
ignoreAllNullRows) {
     this.context = context;
-    this.debug = context.isDebug();
     this.resource = resource;
     this.ignoreAllNullRows = ignoreAllNullRows;
   }
@@ -90,7 +88,7 @@ public class DiskAlignedChunkLoader implements IChunkLoader {
                       resource.isClosed()),
                   timeChunkMetadata.getDeleteIntervalList(),
                   timeChunkMetadata.getStatistics(),
-                  debug);
+                  context);
       List<Chunk> valueChunkList = new ArrayList<>();
       for (IChunkMetadata valueChunkMetadata : 
alignedChunkMetadata.getValueChunkMetadataList()) {
         valueChunkList.add(
@@ -105,7 +103,7 @@ public class DiskAlignedChunkLoader implements IChunkLoader 
{
                             resource.isClosed()),
                         valueChunkMetadata.getDeleteIntervalList(),
                         valueChunkMetadata.getStatistics(),
-                        debug));
+                        context));
       }
 
       long t2 = System.nanoTime();
diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java
index a956f4ef94f..be33428ae65 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/reader/chunk/DiskChunkLoader.java
@@ -44,11 +44,8 @@ public class DiskChunkLoader implements IChunkLoader {
 
   private final TsFileResource resource;
 
-  private final boolean debug;
-
   public DiskChunkLoader(QueryContext context, TsFileResource resource) {
     this.context = context;
-    this.debug = context.isDebug();
     this.resource = resource;
   }
 
@@ -63,7 +60,7 @@ public class DiskChunkLoader implements IChunkLoader {
                 resource.isClosed()),
             chunkMetaData.getDeleteIntervalList(),
             chunkMetaData.getStatistics(),
-            debug);
+            context);
   }
 
   @Override
@@ -86,7 +83,7 @@ public class DiskChunkLoader implements IChunkLoader {
                       resource.isClosed()),
                   chunkMetaData.getDeleteIntervalList(),
                   chunkMetaData.getStatistics(),
-                  debug);
+                  context);
 
       long t2 = System.nanoTime();
       IChunkReader chunkReader = new ChunkReader(chunk, globalTimeFilter);
diff --git 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/ChunkCacheTest.java
 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/ChunkCacheTest.java
index 1783af0b00f..0477b1bf027 100644
--- 
a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/ChunkCacheTest.java
+++ 
b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/buffer/ChunkCacheTest.java
@@ -112,8 +112,7 @@ public class ChunkCacheTest {
                   firstChunkMetadata.getOffsetOfChunkHeader(),
                   true),
               firstChunkMetadata.getDeleteIntervalList(),
-              firstChunkMetadata.getStatistics(),
-              false);
+              firstChunkMetadata.getStatistics());
 
       ChunkMetadata chunkMetadataKey =
           new ChunkMetadata("sensor0", TSDataType.DOUBLE, null, null, 26, new 
DoubleStatistics());
@@ -128,8 +127,7 @@ public class ChunkCacheTest {
                   chunkMetadataKey.getOffsetOfChunkHeader(),
                   true),
               chunkMetadataKey.getDeleteIntervalList(),
-              chunkMetadataKey.getStatistics(),
-              false);
+              chunkMetadataKey.getStatistics());
       Assert.assertEquals(chunk1.getHeader(), chunk2.getHeader());
       Assert.assertEquals(chunk1.getData(), chunk2.getData());
     }
diff --git 
a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/IoTDBIORuntimeException.java
 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/IoTDBIORuntimeException.java
new file mode 100644
index 00000000000..b29c709e5da
--- /dev/null
+++ 
b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/IoTDBIORuntimeException.java
@@ -0,0 +1,14 @@
+package org.apache.iotdb.commons.exception;
+
+import java.io.IOException;
+
+public class IoTDBIORuntimeException extends RuntimeException {
+  public IoTDBIORuntimeException(IOException cause) {
+    super(cause);
+  }
+
+  @Override
+  public synchronized IOException getCause() {
+    return (IOException) super.getCause();
+  }
+}
diff --git a/pom.xml b/pom.xml
index 8738d197a28..90ccc6c0fd2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -166,7 +166,7 @@
         <thrift.version>0.14.1</thrift.version>
         <xz.version>1.9</xz.version>
         <zstd-jni.version>1.5.6-3</zstd-jni.version>
-        <tsfile.version>1.2.0-241108-SNAPSHOT</tsfile.version>
+        <tsfile.version>1.2.0-SNAPSHOT</tsfile.version>
     </properties>
     <!--
     if we claim dependencies in dependencyManagement, then we do not claim


Reply via email to