diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java index 4cc2245fae95..63b74fe9a19d 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java @@ -49,6 +49,7 @@ public class DeltaLakeConfig static final DataSize DEFAULT_DATA_FILE_CACHE_SIZE = DataSize.succinctBytes(Math.floorDiv(Runtime.getRuntime().maxMemory(), 10L)); private Duration metadataCacheTtl = new Duration(5, TimeUnit.MINUTES); + private long metadataCacheMaxSize = 1000; private DataSize dataFileCacheSize = DEFAULT_DATA_FILE_CACHE_SIZE; private Duration dataFileCacheTtl = new Duration(30, TimeUnit.MINUTES); private int domainCompactionThreshold = 100; @@ -87,6 +88,19 @@ public DeltaLakeConfig setMetadataCacheTtl(Duration metadataCacheTtl) return this; } + public long getMetadataCacheMaxSize() + { + return metadataCacheMaxSize; + } + + @Config("delta.metadata.cache-size") + @ConfigDescription("Maximum number of Delta table metadata entries to cache") + public DeltaLakeConfig setMetadataCacheMaxSize(long metadataCacheMaxSize) + { + this.metadataCacheMaxSize = metadataCacheMaxSize; + return this; + } + public DataSize getDataFileCacheSize() { return dataFileCacheSize; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java index 3c4bf1c372de..218fc9ae3ee7 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java @@ -112,6 +112,7 @@ public TransactionLogAccess( tableSnapshots = EvictableCacheBuilder.newBuilder() .expireAfterWrite(deltaLakeConfig.getMetadataCacheTtl().toMillis(), TimeUnit.MILLISECONDS) + .maximumSize(deltaLakeConfig.getMetadataCacheMaxSize()) .shareNothingWhenDisabled() .recordStats() .build(); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java index 30b6adc67d4c..36f3ddcf8e4b 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java @@ -42,6 +42,7 @@ public void testDefaults() .setDataFileCacheSize(DeltaLakeConfig.DEFAULT_DATA_FILE_CACHE_SIZE) .setDataFileCacheTtl(new Duration(30, MINUTES)) .setMetadataCacheTtl(new Duration(5, TimeUnit.MINUTES)) + .setMetadataCacheMaxSize(1000) .setDomainCompactionThreshold(100) .setMaxSplitsPerSecond(Integer.MAX_VALUE) .setMaxOutstandingSplits(1_000) @@ -71,6 +72,7 @@ public void testExplicitPropertyMappings() { Map properties = ImmutableMap.builder() .put("delta.metadata.cache-ttl", "10m") + .put("delta.metadata.cache-size", "10") .put("delta.metadata.live-files.cache-size", "0 MB") .put("delta.metadata.live-files.cache-ttl", "60m") .put("delta.domain-compaction-threshold", "500") @@ -101,6 +103,7 @@ public void testExplicitPropertyMappings() .setDataFileCacheSize(DataSize.succinctBytes(0)) .setDataFileCacheTtl(new Duration(60, MINUTES)) .setMetadataCacheTtl(new Duration(10, TimeUnit.MINUTES)) + .setMetadataCacheMaxSize(10) .setDomainCompactionThreshold(500) .setMaxOutstandingSplits(200) .setMaxSplitsPerSecond(10)