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 2021/08/23 05:16:35 UTC

[iotdb] branch rel/0.12 updated: [To rel/0.12] [ISSUE-3805] OOM caused by Chunk cache (#3806)

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

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


The following commit(s) were added to refs/heads/rel/0.12 by this push:
     new 3194b95  [To rel/0.12] [ISSUE-3805]  OOM caused by Chunk cache (#3806)
3194b95 is described below

commit 3194b95c055ed62eb296af4ac321544a29ed44c3
Author: Xiangwei Wei <34...@users.noreply.github.com>
AuthorDate: Mon Aug 23 13:14:27 2021 +0800

    [To rel/0.12] [ISSUE-3805]  OOM caused by Chunk cache (#3806)
---
 RELEASE_NOTES.md                                   |  1 +
 .../apache/iotdb/db/engine/cache/ChunkCache.java   | 54 ++++----------
 .../db/engine/cache/TimeSeriesMetadataCache.java   | 85 ++++------------------
 3 files changed, 31 insertions(+), 109 deletions(-)

diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md
index 1ad2908..8ff23c3 100644
--- a/RELEASE_NOTES.md
+++ b/RELEASE_NOTES.md
@@ -78,6 +78,7 @@
 * [ISSUE-3545] Fix Time interval value is disorder in group by month
 * [ISSUE-3653] fix Max_time and last return inconsistent result
 * [ISSUE-3690] Memory leaks on the server when cpp client invokes checkTimeseriesExists
+* [ISSUE-3805] OOM caused by Chunk cache
 *  Fix DataMigrationExample OOM if migrate too many timeseries 
 * Handle false positive cases which may cause NPE of tsfile bloom filter 
 * Fix Windows shell error on JDK11 & fix iotdb-env.bat not working
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java
index 07c4ee2..abd3d3d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/ChunkCache.java
@@ -28,7 +28,6 @@ import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Chunk;
 import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
 
-import com.github.benmanes.caffeine.cache.CacheLoader;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.github.benmanes.caffeine.cache.Weigher;
@@ -63,50 +62,25 @@ public class ChunkCache {
         Caffeine.newBuilder()
             .maximumWeight(MEMORY_THRESHOLD_IN_CHUNK_CACHE)
             .weigher(
-                new Weigher<ChunkMetadata, Chunk>() {
-
-                  int count = 0;
-                  int averageSize = 0;
-
-                  @Override
-                  public int weigh(ChunkMetadata chunkMetadata, Chunk chunk) {
-                    int currentSize;
-                    if (count < 10) {
-                      currentSize =
-                          (int)
-                              (RamUsageEstimator.NUM_BYTES_OBJECT_REF
-                                  + RamUsageEstimator.sizeOf(chunk));
-                      averageSize = ((averageSize * count) + currentSize) / (++count);
-                      entryAverageSize.set(averageSize);
-                    } else if (count < 100000) {
-                      count++;
-                      currentSize = averageSize;
-                    } else {
-                      averageSize =
+                (Weigher<ChunkMetadata, Chunk>)
+                    (chunkMetadata, chunk) -> {
+                      int entrySize =
                           (int)
                               (RamUsageEstimator.NUM_BYTES_OBJECT_REF
                                   + RamUsageEstimator.sizeOf(chunk));
-                      count = 1;
-                      currentSize = averageSize;
-                      entryAverageSize.set(averageSize);
-                    }
-                    return currentSize;
-                  }
-                })
+                      return entrySize;
+                    })
             .recordStats()
             .build(
-                new CacheLoader<ChunkMetadata, Chunk>() {
-                  @Override
-                  public Chunk load(ChunkMetadata chunkMetadata) throws Exception {
-                    try {
-                      TsFileSequenceReader reader =
-                          FileReaderManager.getInstance()
-                              .get(chunkMetadata.getFilePath(), chunkMetadata.isClosed());
-                      return reader.readMemChunk(chunkMetadata);
-                    } catch (IOException e) {
-                      logger.error("Something wrong happened in reading {}", chunkMetadata, e);
-                      throw e;
-                    }
+                chunkMetadata -> {
+                  try {
+                    TsFileSequenceReader reader =
+                        FileReaderManager.getInstance()
+                            .get(chunkMetadata.getFilePath(), chunkMetadata.isClosed());
+                    return reader.readMemChunk(chunkMetadata);
+                  } catch (IOException e) {
+                    logger.error("Something wrong happened in reading {}", chunkMetadata, e);
+                    throw e;
                   }
                 });
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
index 4fd205c..9ecf318 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
@@ -24,16 +24,14 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.utils.TestOnly;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
 import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
 
-import com.github.benmanes.caffeine.cache.CacheLoader;
+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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -61,7 +59,7 @@ public class TimeSeriesMetadataCache {
       config.getAllocateMemoryForTimeSeriesMetaDataCache();
   private static final boolean CACHE_ENABLE = config.isMetaDataCacheEnable();
 
-  private final LoadingCache<TimeSeriesMetadataCacheKey, TimeseriesMetadata> lruCache;
+  private final Cache<TimeSeriesMetadataCacheKey, TimeseriesMetadata> lruCache;
 
   private final AtomicLong entryAverageSize = new AtomicLong(0);
 
@@ -78,72 +76,21 @@ public class TimeSeriesMetadataCache {
         Caffeine.newBuilder()
             .maximumWeight(MEMORY_THRESHOLD_IN_TIME_SERIES_METADATA_CACHE)
             .weigher(
-                new Weigher<TimeSeriesMetadataCacheKey, TimeseriesMetadata>() {
-
-                  int count = 0;
-                  int averageSize = 0;
-
-                  @Override
-                  public int weigh(TimeSeriesMetadataCacheKey key, TimeseriesMetadata value) {
-                    int currentSize;
-                    if (count < 10) {
-                      currentSize =
-                          (int)
-                              (RamUsageEstimator.shallowSizeOf(key)
-                                  + RamUsageEstimator.sizeOf(key.device)
-                                  + RamUsageEstimator.sizeOf(key.measurement)
-                                  + RamUsageEstimator.shallowSizeOf(value)
-                                  + RamUsageEstimator.sizeOf(value.getMeasurementId())
-                                  + RamUsageEstimator.shallowSizeOf(value.getStatistics())
-                                  + (((ChunkMetadata) value.getChunkMetadataList().get(0))
-                                              .calculateRamSize()
-                                          + RamUsageEstimator.NUM_BYTES_OBJECT_REF)
-                                      * value.getChunkMetadataList().size()
-                                  + RamUsageEstimator.shallowSizeOf(value.getChunkMetadataList()));
-                      averageSize = ((averageSize * count) + currentSize) / (++count);
-                      entryAverageSize.set(averageSize);
-                    } else if (count < 100000) {
-                      count++;
-                      currentSize = averageSize;
-                    } else {
-                      averageSize =
-                          (int)
-                              (RamUsageEstimator.shallowSizeOf(key)
-                                  + RamUsageEstimator.sizeOf(key.device)
-                                  + RamUsageEstimator.sizeOf(key.measurement)
-                                  + RamUsageEstimator.shallowSizeOf(value)
-                                  + RamUsageEstimator.sizeOf(value.getMeasurementId())
-                                  + RamUsageEstimator.shallowSizeOf(value.getStatistics())
-                                  + (((ChunkMetadata) value.getChunkMetadataList().get(0))
-                                              .calculateRamSize()
-                                          + RamUsageEstimator.NUM_BYTES_OBJECT_REF)
-                                      * value.getChunkMetadataList().size()
-                                  + RamUsageEstimator.shallowSizeOf(value.getChunkMetadataList()));
-                      count = 1;
-                      currentSize = averageSize;
-                      entryAverageSize.set(averageSize);
-                    }
-                    return currentSize;
-                  }
-                })
+                (Weigher<TimeSeriesMetadataCacheKey, TimeseriesMetadata>)
+                    (key, value) ->
+                        (int)
+                            (RamUsageEstimator.shallowSizeOf(key)
+                                + RamUsageEstimator.sizeOf(key.device)
+                                + RamUsageEstimator.sizeOf(key.measurement)
+                                + RamUsageEstimator.shallowSizeOf(value)
+                                + RamUsageEstimator.sizeOf(value.getMeasurementId())
+                                + RamUsageEstimator.shallowSizeOf(value.getStatistics())
+                                + (value.getChunkMetadataList().get(0).calculateRamSize()
+                                        + RamUsageEstimator.NUM_BYTES_OBJECT_REF)
+                                    * value.getChunkMetadataList().size()
+                                + RamUsageEstimator.shallowSizeOf(value.getChunkMetadataList())))
             .recordStats()
-            .build(
-                new CacheLoader<TimeSeriesMetadataCacheKey, TimeseriesMetadata>() {
-                  @Override
-                  public TimeseriesMetadata load(TimeSeriesMetadataCacheKey key) throws Exception {
-                    // bloom filter part
-                    TsFileSequenceReader reader =
-                        FileReaderManager.getInstance().get(key.filePath, true);
-                    BloomFilter bloomFilter = reader.readBloomFilter();
-                    if (bloomFilter != null
-                        && !bloomFilter.contains(
-                            key.device + IoTDBConstant.PATH_SEPARATOR + key.measurement)) {
-                      return null;
-                    }
-                    return reader.readTimeseriesMetadata(
-                        new Path(key.device, key.measurement), false);
-                  }
-                });
+            .build();
   }
 
   public static TimeSeriesMetadataCache getInstance() {