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 2021/08/20 05:21:19 UTC

[iotdb] branch fix_tsm_cache updated (75eb861 -> 327ffe8)

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

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


    from 75eb861  return current entry size directly in TimeseriesMetadataCache
     new 567412f  return current  entry size directly in ChunkMetadataCache and Replace LoadingCache with Cache
     new 327ffe8  add log

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.


Summary of changes:
 .../apache/iotdb/db/engine/cache/ChunkCache.java   | 57 +++++++---------------
 .../db/engine/cache/TimeSeriesMetadataCache.java   | 55 ++++++---------------
 2 files changed, 33 insertions(+), 79 deletions(-)

[iotdb] 02/02: add log

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

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

commit 327ffe872c7d05d455cf81ffaf4253fc314a9b78
Author: Alima777 <wx...@gmail.com>
AuthorDate: Fri Aug 20 13:20:15 2021 +0800

    add log
---
 .../java/org/apache/iotdb/db/engine/cache/ChunkCache.java  | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)

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 f419a82..6b3cdbf 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
@@ -63,10 +63,16 @@ public class ChunkCache {
             .maximumWeight(MEMORY_THRESHOLD_IN_CHUNK_CACHE)
             .weigher(
                 (Weigher<ChunkMetadata, Chunk>)
-                    (chunkMetadata, chunk) ->
-                        (int)
-                            (RamUsageEstimator.NUM_BYTES_OBJECT_REF
-                                + RamUsageEstimator.sizeOf(chunk)))
+                    (chunkMetadata, chunk) -> {
+                      int entrySize =
+                          (int)
+                              (RamUsageEstimator.NUM_BYTES_OBJECT_REF
+                                  + RamUsageEstimator.sizeOf(chunk));
+                      if (entrySize > 1024 * 1024 * 10) {
+                        logger.warn(String.format("Chunk size is %d over 10M", entrySize));
+                      }
+                      return entrySize;
+                    })
             .recordStats()
             .build(
                 chunkMetadata -> {

[iotdb] 01/02: return current entry size directly in ChunkMetadataCache and Replace LoadingCache with Cache

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

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

commit 567412fae8606cfdc78d7b1684fe6a902316c309
Author: Alima777 <wx...@gmail.com>
AuthorDate: Fri Aug 20 13:16:00 2021 +0800

    return current  entry size directly in ChunkMetadataCache and Replace LoadingCache with Cache
---
 .../apache/iotdb/db/engine/cache/ChunkCache.java   | 57 ++++++----------------
 .../db/engine/cache/TimeSeriesMetadataCache.java   | 55 ++++++---------------
 2 files changed, 30 insertions(+), 82 deletions(-)

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..f419a82 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,22 @@ 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 =
-                          (int)
-                              (RamUsageEstimator.NUM_BYTES_OBJECT_REF
-                                  + RamUsageEstimator.sizeOf(chunk));
-                      count = 1;
-                      currentSize = averageSize;
-                      entryAverageSize.set(averageSize);
-                    }
-                    return currentSize;
-                  }
-                })
+                (Weigher<ChunkMetadata, Chunk>)
+                    (chunkMetadata, chunk) ->
+                        (int)
+                            (RamUsageEstimator.NUM_BYTES_OBJECT_REF
+                                + RamUsageEstimator.sizeOf(chunk)))
             .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 fa14776..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,42 +76,21 @@ public class TimeSeriesMetadataCache {
         Caffeine.newBuilder()
             .maximumWeight(MEMORY_THRESHOLD_IN_TIME_SERIES_METADATA_CACHE)
             .weigher(
-                new Weigher<TimeSeriesMetadataCacheKey, TimeseriesMetadata>() {
-
-                  @Override
-                  public int weigh(TimeSeriesMetadataCacheKey key, TimeseriesMetadata value) {
-                    return (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()));
-                  }
-                })
+                (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() {