Skip to content

Commit 0f3ff0a

Browse files
committed
Use shared cache for per transaction file listing cache
This allows to cap cache size independently of query concurrency.
1 parent e9fd487 commit 0f3ff0a

File tree

10 files changed

+182
-50
lines changed

10 files changed

+182
-50
lines changed

docs/src/main/sphinx/connector/hive.rst

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -254,7 +254,8 @@ Hive connector documentation.
254254
- How long a cached directory listing is considered valid.
255255
- ``1m``
256256
* - ``hive.per-transaction-file-status-cache.max-retained-size``
257-
- Maximum retained size of cached file status entries per transaction
257+
- Maximum retained size of all entries in per transaction file status cache.
258+
Retained size limit is shared across all running queries.
258259
- ``100MB``
259260
* - ``hive.rcfile.time-zone``
260261
- Adjusts binary encoded timestamp values to a specific time zone. For

plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/s3select/S3SelectTestHelper.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@
4949
import io.trino.plugin.hive.PropertiesSystemTableProvider;
5050
import io.trino.plugin.hive.aws.athena.PartitionProjectionService;
5151
import io.trino.plugin.hive.fs.FileSystemDirectoryLister;
52+
import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory;
5253
import io.trino.plugin.hive.metastore.HiveMetastoreConfig;
5354
import io.trino.plugin.hive.metastore.HiveMetastoreFactory;
5455
import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore;
@@ -162,6 +163,7 @@ public S3SelectTestHelper(String host,
162163
new DefaultHiveMaterializedViewMetadataFactory(),
163164
SqlStandardAccessControlMetadata::new,
164165
new FileSystemDirectoryLister(),
166+
new TransactionScopeCachingDirectoryListerFactory(hiveConfig),
165167
new PartitionProjectionService(this.hiveConfig, ImmutableMap.of(), new TestingTypeManager()),
166168
true);
167169
transactionManager = new HiveTransactionManager(metadataFactory);

plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java

Lines changed: 7 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -15,14 +15,13 @@
1515

1616
import io.airlift.concurrent.BoundedExecutor;
1717
import io.airlift.json.JsonCodec;
18-
import io.airlift.units.DataSize;
1918
import io.airlift.units.Duration;
2019
import io.trino.filesystem.TrinoFileSystemFactory;
2120
import io.trino.hdfs.HdfsEnvironment;
2221
import io.trino.plugin.base.CatalogName;
2322
import io.trino.plugin.hive.aws.athena.PartitionProjectionService;
2423
import io.trino.plugin.hive.fs.DirectoryLister;
25-
import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryLister;
24+
import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory;
2625
import io.trino.plugin.hive.metastore.HiveMetastoreConfig;
2726
import io.trino.plugin.hive.metastore.HiveMetastoreFactory;
2827
import io.trino.plugin.hive.metastore.SemiTransactionalHiveMetastore;
@@ -77,7 +76,7 @@ public class HiveMetadataFactory
7776
private final Optional<Duration> hiveTransactionHeartbeatInterval;
7877
private final ScheduledExecutorService heartbeatService;
7978
private final DirectoryLister directoryLister;
80-
private final DataSize perTransactionFileStatusCacheMaximumDataSize;
79+
private final TransactionScopeCachingDirectoryListerFactory transactionScopeCachingDirectoryListerFactory;
8180
private final PartitionProjectionService partitionProjectionService;
8281
private final boolean allowTableRename;
8382
private final HiveTimestampPrecision hiveViewsTimestampPrecision;
@@ -103,6 +102,7 @@ public HiveMetadataFactory(
103102
HiveMaterializedViewMetadataFactory hiveMaterializedViewMetadataFactory,
104103
AccessControlMetadataFactory accessControlMetadataFactory,
105104
DirectoryLister directoryLister,
105+
TransactionScopeCachingDirectoryListerFactory transactionScopeCachingDirectoryListerFactory,
106106
PartitionProjectionService partitionProjectionService,
107107
@AllowHiveTableRename boolean allowTableRename)
108108
{
@@ -138,7 +138,7 @@ public HiveMetadataFactory(
138138
hiveMaterializedViewMetadataFactory,
139139
accessControlMetadataFactory,
140140
directoryLister,
141-
hiveConfig.getPerTransactionFileStatusCacheMaxRetainedSize(),
141+
transactionScopeCachingDirectoryListerFactory,
142142
partitionProjectionService,
143143
allowTableRename,
144144
hiveConfig.getTimestampPrecision());
@@ -176,7 +176,7 @@ public HiveMetadataFactory(
176176
HiveMaterializedViewMetadataFactory hiveMaterializedViewMetadataFactory,
177177
AccessControlMetadataFactory accessControlMetadataFactory,
178178
DirectoryLister directoryLister,
179-
DataSize perTransactionFileStatusCacheMaximumDataSize,
179+
TransactionScopeCachingDirectoryListerFactory transactionScopeCachingDirectoryListerFactory,
180180
PartitionProjectionService partitionProjectionService,
181181
boolean allowTableRename,
182182
HiveTimestampPrecision hiveViewsTimestampPrecision)
@@ -219,7 +219,7 @@ public HiveMetadataFactory(
219219
this.maxPartitionDropsPerQuery = maxPartitionDropsPerQuery;
220220
this.heartbeatService = requireNonNull(heartbeatService, "heartbeatService is null");
221221
this.directoryLister = requireNonNull(directoryLister, "directoryLister is null");
222-
this.perTransactionFileStatusCacheMaximumDataSize = requireNonNull(perTransactionFileStatusCacheMaximumDataSize, "perTransactionFileStatusCacheMaximumDataSize is null");
222+
this.transactionScopeCachingDirectoryListerFactory = requireNonNull(transactionScopeCachingDirectoryListerFactory, "transactionScopeCachingDirectoryListerFactory is null");
223223
this.partitionProjectionService = requireNonNull(partitionProjectionService, "partitionProjectionService is null");
224224
this.allowTableRename = allowTableRename;
225225
this.hiveViewsTimestampPrecision = requireNonNull(hiveViewsTimestampPrecision, "hiveViewsTimestampPrecision is null");
@@ -231,14 +231,7 @@ public TransactionalMetadata create(ConnectorIdentity identity, boolean autoComm
231231
HiveMetastoreClosure hiveMetastoreClosure = new HiveMetastoreClosure(
232232
memoizeMetastore(metastoreFactory.createMetastore(Optional.of(identity)), perTransactionCacheMaximumSize)); // per-transaction cache
233233

234-
DirectoryLister directoryLister;
235-
if (perTransactionFileStatusCacheMaximumDataSize.toBytes() > 0) {
236-
directoryLister = new TransactionScopeCachingDirectoryLister(this.directoryLister, perTransactionFileStatusCacheMaximumDataSize);
237-
}
238-
else {
239-
directoryLister = this.directoryLister;
240-
}
241-
234+
DirectoryLister directoryLister = transactionScopeCachingDirectoryListerFactory.get(this.directoryLister);
242235
SemiTransactionalHiveMetastore metastore = new SemiTransactionalHiveMetastore(
243236
hdfsEnvironment,
244237
hiveMetastoreClosure,

plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import io.trino.hdfs.TrinoFileSystemCacheStats;
2626
import io.trino.plugin.base.CatalogName;
2727
import io.trino.plugin.hive.fs.CachingDirectoryLister;
28+
import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory;
2829
import io.trino.plugin.hive.line.CsvFileWriterFactory;
2930
import io.trino.plugin.hive.line.CsvPageSourceFactory;
3031
import io.trino.plugin.hive.line.JsonFileWriterFactory;
@@ -106,6 +107,7 @@ public void configure(Binder binder)
106107
.setDefault().to(DefaultHiveMaterializedViewMetadataFactory.class).in(Scopes.SINGLETON);
107108
newOptionalBinder(binder, TransactionalMetadataFactory.class)
108109
.setDefault().to(HiveMetadataFactory.class).in(Scopes.SINGLETON);
110+
binder.bind(TransactionScopeCachingDirectoryListerFactory.class).in(Scopes.SINGLETON);
109111
binder.bind(HiveTransactionManager.class).in(Scopes.SINGLETON);
110112
binder.bind(ConnectorSplitManager.class).to(HiveSplitManager.class).in(Scopes.SINGLETON);
111113
newExporter(binder).export(ConnectorSplitManager.class).as(generator -> generator.generatedNameOf(HiveSplitManager.class));
Lines changed: 72 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,72 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package io.trino.plugin.hive.fs;
15+
16+
import java.util.Objects;
17+
18+
import static com.google.common.base.MoreObjects.toStringHelper;
19+
import static io.airlift.slice.SizeOf.instanceSize;
20+
import static java.util.Objects.requireNonNull;
21+
22+
public class TransactionDirectoryListingCacheKey
23+
{
24+
private static final long INSTANCE_SIZE = instanceSize(TransactionDirectoryListingCacheKey.class);
25+
26+
private final long transactionId;
27+
private final DirectoryListingCacheKey key;
28+
29+
public TransactionDirectoryListingCacheKey(long transactionId, DirectoryListingCacheKey key)
30+
{
31+
this.transactionId = transactionId;
32+
this.key = requireNonNull(key, "key is null");
33+
}
34+
35+
public DirectoryListingCacheKey getKey()
36+
{
37+
return key;
38+
}
39+
40+
public long getRetainedSizeInBytes()
41+
{
42+
return INSTANCE_SIZE + key.getRetainedSizeInBytes();
43+
}
44+
45+
@Override
46+
public boolean equals(Object o)
47+
{
48+
if (this == o) {
49+
return true;
50+
}
51+
if (o == null || getClass() != o.getClass()) {
52+
return false;
53+
}
54+
TransactionDirectoryListingCacheKey that = (TransactionDirectoryListingCacheKey) o;
55+
return transactionId == that.transactionId && key.equals(that.key);
56+
}
57+
58+
@Override
59+
public int hashCode()
60+
{
61+
return Objects.hash(transactionId, key);
62+
}
63+
64+
@Override
65+
public String toString()
66+
{
67+
return toStringHelper(this)
68+
.add("transactionId", transactionId)
69+
.add("key", key)
70+
.toString();
71+
}
72+
}

plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TransactionScopeCachingDirectoryLister.java

Lines changed: 25 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,6 @@
1616
import com.google.common.annotations.VisibleForTesting;
1717
import com.google.common.cache.Cache;
1818
import com.google.common.util.concurrent.UncheckedExecutionException;
19-
import io.airlift.units.DataSize;
20-
import io.trino.collect.cache.EvictableCacheBuilder;
2119
import io.trino.plugin.hive.metastore.Partition;
2220
import io.trino.plugin.hive.metastore.Storage;
2321
import io.trino.plugin.hive.metastore.Table;
@@ -40,9 +38,10 @@
4038
import static com.google.common.base.Preconditions.checkState;
4139
import static com.google.common.base.Throwables.throwIfInstanceOf;
4240
import static com.google.common.base.Throwables.throwIfUnchecked;
41+
import static com.google.common.collect.ImmutableList.toImmutableList;
4342
import static io.airlift.slice.SizeOf.instanceSize;
4443
import static io.airlift.slice.SizeOf.sizeOfObjectArray;
45-
import static java.lang.Math.toIntExact;
44+
import static io.trino.plugin.hive.fs.DirectoryListingCacheKey.allKeysWithPath;
4645
import static java.util.Collections.synchronizedList;
4746
import static java.util.Objects.requireNonNull;
4847

@@ -55,42 +54,34 @@
5554
public class TransactionScopeCachingDirectoryLister
5655
implements DirectoryLister
5756
{
57+
private final long transactionId;
5858
//TODO use a cache key based on Path & SchemaTableName and iterate over the cache keys
5959
// to deal more efficiently with cache invalidation scenarios for partitioned tables.
60-
private final Cache<DirectoryListingCacheKey, FetchingValueHolder> cache;
60+
private final Cache<TransactionDirectoryListingCacheKey, FetchingValueHolder> cache;
6161
private final DirectoryLister delegate;
6262

63-
public TransactionScopeCachingDirectoryLister(DirectoryLister delegate, DataSize maxSize)
63+
public TransactionScopeCachingDirectoryLister(DirectoryLister delegate, long transactionId, Cache<TransactionDirectoryListingCacheKey, FetchingValueHolder> cache)
6464
{
65-
this(delegate, maxSize, Optional.empty());
66-
}
67-
68-
@VisibleForTesting
69-
TransactionScopeCachingDirectoryLister(DirectoryLister delegate, DataSize maxSize, Optional<Integer> concurrencyLevel)
70-
{
71-
EvictableCacheBuilder<DirectoryListingCacheKey, FetchingValueHolder> cacheBuilder = EvictableCacheBuilder.newBuilder()
72-
.maximumWeight(maxSize.toBytes())
73-
.weigher((key, value) -> toIntExact(key.getRetainedSizeInBytes() + value.getRetainedSizeInBytes()));
74-
concurrencyLevel.ifPresent(cacheBuilder::concurrencyLevel);
75-
this.cache = cacheBuilder.build();
7665
this.delegate = requireNonNull(delegate, "delegate is null");
66+
this.transactionId = transactionId;
67+
this.cache = requireNonNull(cache, "cache is null");
7768
}
7869

7970
@Override
8071
public RemoteIterator<TrinoFileStatus> list(FileSystem fs, Table table, Path path)
8172
throws IOException
8273
{
83-
return listInternal(fs, table, new DirectoryListingCacheKey(path.toString(), false));
74+
return listInternal(fs, table, new TransactionDirectoryListingCacheKey(transactionId, new DirectoryListingCacheKey(path.toString(), false)));
8475
}
8576

8677
@Override
8778
public RemoteIterator<TrinoFileStatus> listFilesRecursively(FileSystem fs, Table table, Path path)
8879
throws IOException
8980
{
90-
return listInternal(fs, table, new DirectoryListingCacheKey(path.toString(), true));
81+
return listInternal(fs, table, new TransactionDirectoryListingCacheKey(transactionId, new DirectoryListingCacheKey(path.toString(), true)));
9182
}
9283

93-
private RemoteIterator<TrinoFileStatus> listInternal(FileSystem fs, Table table, DirectoryListingCacheKey cacheKey)
84+
private RemoteIterator<TrinoFileStatus> listInternal(FileSystem fs, Table table, TransactionDirectoryListingCacheKey cacheKey)
9485
throws IOException
9586
{
9687
FetchingValueHolder cachedValueHolder;
@@ -101,7 +92,7 @@ private RemoteIterator<TrinoFileStatus> listInternal(FileSystem fs, Table table,
10192
Throwable throwable = e.getCause();
10293
throwIfInstanceOf(throwable, IOException.class);
10394
throwIfUnchecked(throwable);
104-
throw new RuntimeException("Failed to list directory: " + cacheKey.getPath(), throwable);
95+
throw new RuntimeException("Failed to list directory: " + cacheKey.getKey().getPath(), throwable);
10596
}
10697

10798
if (cachedValueHolder.isFullyCached()) {
@@ -111,21 +102,23 @@ private RemoteIterator<TrinoFileStatus> listInternal(FileSystem fs, Table table,
111102
return cachingRemoteIterator(cachedValueHolder, cacheKey);
112103
}
113104

114-
private RemoteIterator<TrinoFileStatus> createListingRemoteIterator(FileSystem fs, Table table, DirectoryListingCacheKey cacheKey)
105+
private RemoteIterator<TrinoFileStatus> createListingRemoteIterator(FileSystem fs, Table table, TransactionDirectoryListingCacheKey cacheKey)
115106
throws IOException
116107
{
117-
if (cacheKey.isRecursiveFilesOnly()) {
118-
return delegate.listFilesRecursively(fs, table, new Path(cacheKey.getPath()));
108+
if (cacheKey.getKey().isRecursiveFilesOnly()) {
109+
return delegate.listFilesRecursively(fs, table, new Path(cacheKey.getKey().getPath()));
119110
}
120-
return delegate.list(fs, table, new Path(cacheKey.getPath()));
111+
return delegate.list(fs, table, new Path(cacheKey.getKey().getPath()));
121112
}
122113

123114
@Override
124115
public void invalidate(Table table)
125116
{
126117
if (isLocationPresent(table.getStorage())) {
127118
if (table.getPartitionColumns().isEmpty()) {
128-
cache.invalidateAll(DirectoryListingCacheKey.allKeysWithPath(new Path(table.getStorage().getLocation())));
119+
cache.invalidateAll(allKeysWithPath(new Path(table.getStorage().getLocation())).stream()
120+
.map(key -> new TransactionDirectoryListingCacheKey(transactionId, key))
121+
.collect(toImmutableList()));
129122
}
130123
else {
131124
// a partitioned table can have multiple paths in cache
@@ -139,12 +132,14 @@ public void invalidate(Table table)
139132
public void invalidate(Partition partition)
140133
{
141134
if (isLocationPresent(partition.getStorage())) {
142-
cache.invalidateAll(DirectoryListingCacheKey.allKeysWithPath(new Path(partition.getStorage().getLocation())));
135+
cache.invalidateAll(allKeysWithPath(new Path(partition.getStorage().getLocation())).stream()
136+
.map(key -> new TransactionDirectoryListingCacheKey(transactionId, key))
137+
.collect(toImmutableList()));
143138
}
144139
delegate.invalidate(partition);
145140
}
146141

147-
private RemoteIterator<TrinoFileStatus> cachingRemoteIterator(FetchingValueHolder cachedValueHolder, DirectoryListingCacheKey cacheKey)
142+
private RemoteIterator<TrinoFileStatus> cachingRemoteIterator(FetchingValueHolder cachedValueHolder, TransactionDirectoryListingCacheKey cacheKey)
148143
{
149144
return new RemoteIterator<>()
150145
{
@@ -183,11 +178,11 @@ public TrinoFileStatus next()
183178
@VisibleForTesting
184179
boolean isCached(Path path)
185180
{
186-
return isCached(new DirectoryListingCacheKey(path.toString(), false));
181+
return isCached(new TransactionDirectoryListingCacheKey(transactionId, new DirectoryListingCacheKey(path.toString(), false)));
187182
}
188183

189184
@VisibleForTesting
190-
boolean isCached(DirectoryListingCacheKey cacheKey)
185+
boolean isCached(TransactionDirectoryListingCacheKey cacheKey)
191186
{
192187
FetchingValueHolder cached = cache.getIfPresent(cacheKey);
193188
return cached != null && cached.isFullyCached();
@@ -199,7 +194,7 @@ private static boolean isLocationPresent(Storage storage)
199194
return storage.getOptionalLocation().isPresent() && !storage.getLocation().isEmpty();
200195
}
201196

202-
private static class FetchingValueHolder
197+
static class FetchingValueHolder
203198
{
204199
private static final long ATOMIC_LONG_SIZE = instanceSize(AtomicLong.class);
205200
private static final long INSTANCE_SIZE = instanceSize(FetchingValueHolder.class);

0 commit comments

Comments
 (0)