diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 90c6050ad43a..39de44a72ef0 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -362,7 +362,8 @@ jobs: !:trino-faulttolerant-tests, !:trino-filesystem, !:trino-filesystem-alluxio, - !:trino-filesystem-cache-alluxio, + !:trino-blob-cache-alluxio, + !:trino-blob-cache-memory, !:trino-filesystem-azure, !:trino-filesystem-gcs, !:trino-filesystem-manager, @@ -466,12 +467,13 @@ jobs: - lib/trino-filesystem - lib/trino-filesystem-azure - lib/trino-filesystem-alluxio - - lib/trino-filesystem-cache-alluxio - lib/trino-filesystem-gcs - lib/trino-filesystem-manager - lib/trino-filesystem-s3 - lib/trino-hdfs - lib/trino-hive-formats + - plugin/trino-blob-cache-alluxio + - plugin/trino-blob-cache-memory - { modules: core/trino-main } - { modules: lib/trino-filesystem-azure, profile: cloud-tests } - { modules: lib/trino-filesystem-gcs, profile: cloud-tests } diff --git a/core/docker/default/etc/cache-manager-memory.properties b/core/docker/default/etc/cache-manager-memory.properties new file mode 100644 index 000000000000..16939d877700 --- /dev/null +++ b/core/docker/default/etc/cache-manager-memory.properties @@ -0,0 +1,3 @@ +cache-manager.name=memory +fs.memory-cache.max-size=2GB +fs.memory-cache.ttl=24h diff --git a/core/docker/default/etc/config.properties b/core/docker/default/etc/config.properties index 559b9a37ea3e..6835eaf8667d 100644 --- a/core/docker/default/etc/config.properties +++ b/core/docker/default/etc/config.properties @@ -4,3 +4,4 @@ node-scheduler.include-coordinator=true http-server.http.port=8080 discovery.uri=http://localhost:8080 catalog.management=${ENV:CATALOG_MANAGEMENT} +cache-manager.config-files=etc/cache-manager-memory.properties diff --git a/core/trino-main/src/main/java/io/trino/cache/CacheManagerConfig.java b/core/trino-main/src/main/java/io/trino/cache/CacheManagerConfig.java new file mode 100644 index 000000000000..c4df0f520e50 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/cache/CacheManagerConfig.java @@ -0,0 +1,45 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.cache; + +import com.google.common.collect.ImmutableList; +import io.airlift.configuration.Config; +import io.airlift.configuration.validation.FileExists; + +import java.io.File; +import java.util.List; +import java.util.stream.Stream; + +import static com.google.common.collect.ImmutableList.toImmutableList; + +public class CacheManagerConfig +{ + private List cacheManagerConfigFiles = ImmutableList.of(); + + public List<@FileExists File> getCacheManagerConfigFiles() + { + return cacheManagerConfigFiles; + } + + @Config("cache-manager.config-files") + public CacheManagerConfig setCacheManagerConfigFiles(String cacheManagerConfigFiles) + { + this.cacheManagerConfigFiles = Stream.of(cacheManagerConfigFiles.split(",")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .map(File::new) + .collect(toImmutableList()); + return this; + } +} diff --git a/core/trino-main/src/main/java/io/trino/cache/CacheManagerContextInstance.java b/core/trino-main/src/main/java/io/trino/cache/CacheManagerContextInstance.java new file mode 100644 index 000000000000..c5052c0178ca --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/cache/CacheManagerContextInstance.java @@ -0,0 +1,45 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.cache; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Tracer; +import io.trino.spi.cache.CacheManagerContext; + +import static java.util.Objects.requireNonNull; + +public class CacheManagerContextInstance + implements CacheManagerContext +{ + private final OpenTelemetry openTelemetry; + private final Tracer tracer; + + public CacheManagerContextInstance(OpenTelemetry openTelemetry, Tracer tracer) + { + this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); + this.tracer = requireNonNull(tracer, "tracer is null"); + } + + @Override + public OpenTelemetry getOpenTelemetry() + { + return openTelemetry; + } + + @Override + public Tracer getTracer() + { + return tracer; + } +} diff --git a/core/trino-main/src/main/java/io/trino/cache/CacheManagerModule.java b/core/trino-main/src/main/java/io/trino/cache/CacheManagerModule.java new file mode 100644 index 000000000000..be38023b66fe --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/cache/CacheManagerModule.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.cache; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class CacheManagerModule + implements Module +{ + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(CacheManagerConfig.class); + binder.bind(CacheManagerRegistry.class).in(Scopes.SINGLETON); + } +} diff --git a/core/trino-main/src/main/java/io/trino/cache/CacheManagerRegistry.java b/core/trino-main/src/main/java/io/trino/cache/CacheManagerRegistry.java new file mode 100644 index 000000000000..13717343b977 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/cache/CacheManagerRegistry.java @@ -0,0 +1,175 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.cache; + +import com.google.inject.Inject; +import io.airlift.configuration.secrets.SecretsResolver; +import io.airlift.log.Logger; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Tracer; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobCacheManager; +import io.trino.spi.cache.BlobCacheManagerFactory; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; +import io.trino.spi.cache.CacheLatency; +import io.trino.spi.cache.ConnectorCacheFactory; +import io.trino.spi.cache.PassThroughBlob; +import io.trino.spi.catalog.CatalogName; +import io.trino.spi.classloader.ThreadContextClassLoader; +import jakarta.annotation.PreDestroy; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static io.airlift.configuration.ConfigurationLoader.loadPropertiesFrom; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class CacheManagerRegistry +{ + private static final Logger log = Logger.get(CacheManagerRegistry.class); + + private static final String CACHE_MANAGER_NAME_PROPERTY = "cache-manager.name"; + + private final OpenTelemetry openTelemetry; + private final Tracer tracer; + private final SecretsResolver secretsResolver; + private final List configFiles; + + private final Map blobCacheFactories = new ConcurrentHashMap<>(); + private final Map blobCacheManagers = new ConcurrentHashMap<>(); + + @Inject + public CacheManagerRegistry( + OpenTelemetry openTelemetry, + Tracer tracer, + SecretsResolver secretsResolver, + CacheManagerConfig config) + { + this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); + this.tracer = requireNonNull(tracer, "tracer is null"); + this.secretsResolver = requireNonNull(secretsResolver, "secretsResolver is null"); + this.configFiles = List.copyOf(config.getCacheManagerConfigFiles()); + } + + public void addBlobCacheManagerFactory(BlobCacheManagerFactory factory) + { + requireNonNull(factory, "factory is null"); + if (blobCacheFactories.putIfAbsent(factory.getName(), factory) != null) { + throw new IllegalArgumentException(format("Blob cache manager factory '%s' is already registered", factory.getName())); + } + } + + public synchronized void loadCacheManagers() + { + for (File configFile : configFiles) { + Map properties = loadProperties(configFile); + String name = properties.remove(CACHE_MANAGER_NAME_PROPERTY); + checkArgument(!isNullOrEmpty(name), "Cache manager configuration %s does not contain %s", configFile, CACHE_MANAGER_NAME_PROPERTY); + loadBlobCacheManager(name, properties); + } + } + + public synchronized void loadBlobCacheManager(String name, Map properties) + { + log.info("-- Loading blob cache manager %s --", name); + + BlobCacheManagerFactory factory = blobCacheFactories.get(name); + checkArgument(factory != null, "Blob cache manager factory '%s' is not registered. Available factories: %s", name, blobCacheFactories.keySet()); + + CacheLatency tier = factory.latency(); + if (blobCacheManagers.containsKey(tier)) { + throw new IllegalStateException(format("Blob cache manager for tier %s is already loaded", tier)); + } + + BlobCacheManager manager; + try (ThreadContextClassLoader _ = new ThreadContextClassLoader(factory.getClass().getClassLoader())) { + manager = factory.create(secretsResolver.getResolvedConfiguration(properties), new CacheManagerContextInstance(openTelemetry, tracer)); + } + + blobCacheManagers.put(tier, manager); + log.info("-- Loaded blob cache manager %s for tier %s --", name, tier); + } + + public ConnectorCacheFactory createConnectorCacheFactory(CatalogName catalog) + { + requireNonNull(catalog, "catalog is null"); + return tier -> { + BlobCacheManager manager = blobCacheManagers.get(tier); + if (manager == null) { + log.warn("Catalog %s requested blob cache manager tier %s but none registered, using noop", catalog, tier); + return new NoopBlobCache(); + } + log.debug("Created new blob cache on tier %s for catalog %s", tier, catalog); + return manager.createBlobCache(catalog); + }; + } + + public void drop(CatalogName catalog) + { + for (Map.Entry entry : blobCacheManagers.entrySet()) { + log.info("Dropping blob cache with latency %s for catalog %s", entry.getKey(), catalog); + entry.getValue().drop(catalog); + } + } + + @PreDestroy + public void shutdown() + { + for (Map.Entry entry : blobCacheManagers.entrySet()) { + try { + entry.getValue().shutdown(); + } + catch (Throwable t) { + log.error(t, "Error shutting down blob cache manager for tier %s", entry.getKey()); + } + } + } + + private static Map loadProperties(File configFile) + { + try { + return new HashMap<>(loadPropertiesFrom(configFile.getPath())); + } + catch (IOException e) { + throw new UncheckedIOException("Failed to read configuration file: " + configFile, e); + } + } + + private static class NoopBlobCache + implements BlobCache + { + @Override + public Blob get(CacheKey key, BlobSource source) + { + return new PassThroughBlob(source); + } + + @Override + public void invalidate(CacheKey key) {} + + @Override + public void invalidate(Collection keys) {} + } +} diff --git a/core/trino-main/src/main/java/io/trino/connector/ConnectorContextInstance.java b/core/trino-main/src/main/java/io/trino/connector/ConnectorContextInstance.java index 48f58bbcfb3f..ef07ddb77d12 100644 --- a/core/trino-main/src/main/java/io/trino/connector/ConnectorContextInstance.java +++ b/core/trino-main/src/main/java/io/trino/connector/ConnectorContextInstance.java @@ -19,6 +19,7 @@ import io.trino.spi.PageIndexerFactory; import io.trino.spi.PageSorter; import io.trino.spi.VersionEmbedder; +import io.trino.spi.cache.ConnectorCacheFactory; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.MetadataProvider; import io.trino.spi.function.FunctionBundleFactory; @@ -38,6 +39,7 @@ public class ConnectorContextInstance private final PageSorter pageSorter; private final PageIndexerFactory pageIndexerFactory; private final FunctionBundleFactory functionBundleFactory; + private final ConnectorCacheFactory cacheFactory; public ConnectorContextInstance( OpenTelemetry openTelemetry, @@ -48,7 +50,8 @@ public ConnectorContextInstance( MetadataProvider metadataProvider, PageSorter pageSorter, PageIndexerFactory pageIndexerFactory, - FunctionBundleFactory functionBundleFactory) + FunctionBundleFactory functionBundleFactory, + ConnectorCacheFactory cacheFactory) { this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); this.tracer = requireNonNull(tracer, "tracer is null"); @@ -59,6 +62,7 @@ public ConnectorContextInstance( this.pageSorter = requireNonNull(pageSorter, "pageSorter is null"); this.pageIndexerFactory = requireNonNull(pageIndexerFactory, "pageIndexerFactory is null"); this.functionBundleFactory = requireNonNull(functionBundleFactory, "functionBundleFactory is null"); + this.cacheFactory = requireNonNull(cacheFactory, "cacheFactory is null"); } @Override @@ -114,4 +118,10 @@ public FunctionBundleFactory getFunctionBundleFactory() { return functionBundleFactory; } + + @Override + public ConnectorCacheFactory getCacheFactory() + { + return cacheFactory; + } } diff --git a/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java b/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java index 3ba2fc4167aa..89f06274c8dc 100644 --- a/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/CoordinatorDynamicCatalogManager.java @@ -20,6 +20,7 @@ import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import io.airlift.log.Logger; +import io.trino.cache.CacheManagerRegistry; import io.trino.connector.system.GlobalSystemConnector; import io.trino.metadata.Catalog; import io.trino.metadata.CatalogManager; @@ -72,6 +73,7 @@ private enum State private final CatalogStore catalogStore; private final CatalogFactory catalogFactory; + private final CacheManagerRegistry cacheManagerRegistry; private final Executor executor; private final Object catalogsUpdateLock = new Object(); @@ -90,10 +92,11 @@ private enum State private State state = State.CREATED; @Inject - public CoordinatorDynamicCatalogManager(CatalogStore catalogStore, CatalogFactory catalogFactory, @ForStartup Executor executor) + public CoordinatorDynamicCatalogManager(CatalogStore catalogStore, CatalogFactory catalogFactory, CacheManagerRegistry cacheManagerRegistry, @ForStartup Executor executor) { this.catalogStore = requireNonNull(catalogStore, "catalogStore is null"); this.catalogFactory = requireNonNull(catalogFactory, "catalogFactory is null"); + this.cacheManagerRegistry = requireNonNull(cacheManagerRegistry, "cacheManagerRegistry is null"); this.executor = requireNonNull(executor, "executor is null"); } @@ -312,6 +315,9 @@ public void dropCatalog(CatalogName catalogName, boolean exists) catalogStore.removeCatalog(catalogName); removed = activeCatalogs.remove(catalogName) != null; + if (removed) { + cacheManagerRegistry.drop(catalogName); + } } if (!removed && !exists) { diff --git a/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java b/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java index 28eb14ee30f1..30811b106003 100644 --- a/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java +++ b/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java @@ -18,6 +18,7 @@ import io.airlift.configuration.secrets.SecretsResolver; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; +import io.trino.cache.CacheManagerRegistry; import io.trino.connector.informationschema.InformationSchemaConnector; import io.trino.connector.system.SystemConnector; import io.trino.connector.system.SystemTablesProvider; @@ -73,6 +74,7 @@ public class DefaultCatalogFactory private final ConcurrentMap connectorFactories = new ConcurrentHashMap<>(); private final SecretsResolver secretsResolver; + private final CacheManagerRegistry cacheManagerRegistry; @Inject public DefaultCatalogFactory( @@ -88,7 +90,8 @@ public DefaultCatalogFactory( TypeManager typeManager, NodeSchedulerConfig nodeSchedulerConfig, OptimizerConfig optimizerConfig, - SecretsResolver secretsResolver) + SecretsResolver secretsResolver, + CacheManagerRegistry cacheManagerRegistry) { this.metadata = requireNonNull(metadata, "metadata is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); @@ -103,6 +106,7 @@ public DefaultCatalogFactory( this.schedulerIncludeCoordinator = nodeSchedulerConfig.isIncludeCoordinator(); this.maxPrefetchedInformationSchemaPrefixes = optimizerConfig.getMaxPrefetchedInformationSchemaPrefixes(); this.secretsResolver = requireNonNull(secretsResolver, "secretsResolver is null"); + this.cacheManagerRegistry = requireNonNull(cacheManagerRegistry, "cacheManagerRegistry is null"); } @Override @@ -194,7 +198,8 @@ private Connector createConnector(CatalogName catalogName, ConnectorFactory conn new InternalMetadataProvider(metadata, typeManager), pageSorter, pageIndexerFactory, - new InternalFunctionBundleFactory()); + new InternalFunctionBundleFactory(), + cacheManagerRegistry.createConnectorCacheFactory(catalogName)); try (ThreadContextClassLoader _ = new ThreadContextClassLoader(connectorFactory.getClass().getClassLoader())) { // TODO: connector factory should take CatalogName diff --git a/core/trino-main/src/main/java/io/trino/server/PluginManager.java b/core/trino-main/src/main/java/io/trino/server/PluginManager.java index 214c9cbe658f..e7f09def4ad8 100644 --- a/core/trino-main/src/main/java/io/trino/server/PluginManager.java +++ b/core/trino-main/src/main/java/io/trino/server/PluginManager.java @@ -17,6 +17,7 @@ import com.google.errorprone.annotations.ThreadSafe; import com.google.inject.Inject; import io.airlift.log.Logger; +import io.trino.cache.CacheManagerRegistry; import io.trino.connector.CatalogFactory; import io.trino.connector.CatalogStoreManager; import io.trino.eventlistener.EventListenerManager; @@ -37,6 +38,7 @@ import io.trino.server.security.PasswordAuthenticatorManager; import io.trino.spi.Plugin; import io.trino.spi.block.BlockEncoding; +import io.trino.spi.cache.BlobCacheManagerFactory; import io.trino.spi.catalog.CatalogStoreFactory; import io.trino.spi.classloader.ThreadContextClassLoader; import io.trino.spi.connector.ConnectorFactory; @@ -94,6 +96,7 @@ public class PluginManager private final GroupProviderManager groupProviderManager; private final ExchangeManagerRegistry exchangeManagerRegistry; private final SpoolingManagerRegistry spoolingManagerRegistry; + private final CacheManagerRegistry cacheManagerRegistry; private final SessionPropertyDefaults sessionPropertyDefaults; private final TypeRegistry typeRegistry; private final BlockEncodingManager blockEncodingManager; @@ -119,7 +122,8 @@ public PluginManager( BlockEncodingManager blockEncodingManager, HandleResolver handleResolver, ExchangeManagerRegistry exchangeManagerRegistry, - SpoolingManagerRegistry spoolingManagerRegistry) + SpoolingManagerRegistry spoolingManagerRegistry, + CacheManagerRegistry cacheManagerRegistry) { this.pluginsProvider = requireNonNull(pluginsProvider, "pluginsProvider is null"); this.catalogStoreManager = requireNonNull(catalogStoreManager, "catalogStoreManager is null"); @@ -139,6 +143,7 @@ public PluginManager( this.handleResolver = requireNonNull(handleResolver, "handleResolver is null"); this.exchangeManagerRegistry = requireNonNull(exchangeManagerRegistry, "exchangeManagerRegistry is null"); this.spoolingManagerRegistry = requireNonNull(spoolingManagerRegistry, "spoolingManagerRegistry is null"); + this.cacheManagerRegistry = requireNonNull(cacheManagerRegistry, "cacheManagerRegistry is null"); } @Override @@ -286,6 +291,11 @@ private void installPluginInternal(Plugin plugin) log.info("Registering spooling manager %s", spoolingManagerFactory.getName()); spoolingManagerRegistry.addSpoolingManagerFactory(spoolingManagerFactory); } + + for (BlobCacheManagerFactory factory : plugin.getBlobCacheManagerFactories()) { + log.info("Registering blob cache manager %s", factory.getName()); + cacheManagerRegistry.addBlobCacheManagerFactory(factory); + } } public static PluginClassLoader createClassLoader(String pluginName, List urls) diff --git a/core/trino-main/src/main/java/io/trino/server/Server.java b/core/trino-main/src/main/java/io/trino/server/Server.java index 931cc62abc79..fd2905505c64 100644 --- a/core/trino-main/src/main/java/io/trino/server/Server.java +++ b/core/trino-main/src/main/java/io/trino/server/Server.java @@ -34,6 +34,8 @@ import io.airlift.openmetrics.JmxOpenMetricsModule; import io.airlift.tracing.TracingModule; import io.airlift.units.Duration; +import io.trino.cache.CacheManagerModule; +import io.trino.cache.CacheManagerRegistry; import io.trino.connector.CatalogManagerModule; import io.trino.connector.CatalogStoreManager; import io.trino.connector.ConnectorServicesProvider; @@ -105,6 +107,7 @@ private void doStart(String trinoVersion) new AccessControlModule(), new EventListenerModule(), new ExchangeManagerModule(), + new CacheManagerModule(), new CatalogManagerModule(), new TransactionManagerModule(), new NodeManagerModule(trinoVersion), @@ -129,6 +132,8 @@ private void doStart(String trinoVersion) logLocation(log, "Etc directory", Path.of("etc")); injector.getInstance(PluginInstaller.class).loadPlugins(); + // Caches can be requested for initial catalogs so we need to wire these first + injector.getInstance(CacheManagerRegistry.class).loadCacheManagers(); var catalogStoreManager = injector.getInstance(Key.get(new TypeLiteral>() {})); catalogStoreManager.ifPresent(CatalogStoreManager::loadConfiguredCatalogStore); diff --git a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java index b7de887b42f5..d7d5dde76471 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java +++ b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java @@ -41,6 +41,8 @@ import io.opentelemetry.sdk.trace.SpanProcessor; import io.trino.Session; import io.trino.SystemSessionPropertiesProvider; +import io.trino.cache.CacheManagerConfig; +import io.trino.cache.CacheManagerRegistry; import io.trino.connector.CatalogHandle; import io.trino.connector.CatalogManagerConfig.CatalogMangerKind; import io.trino.connector.CatalogManagerModule; @@ -216,6 +218,7 @@ public static Builder builder() private final boolean coordinator; private final FailureInjector failureInjector; private final ExchangeManagerRegistry exchangeManagerRegistry; + private final CacheManagerRegistry cacheManagerRegistry; private final SpoolingManagerRegistry spoolingManagerRegistry; public static class TestShutdownAction @@ -321,6 +324,7 @@ private TestingTrinoServer( binder.bind(NodeVersion.class).toInstance(new NodeVersion(VERSION)); binder.bind(EventListenerConfig.class).in(Scopes.SINGLETON); binder.bind(ExchangeManagerConfig.class).in(Scopes.SINGLETON); + binder.bind(CacheManagerConfig.class).in(Scopes.SINGLETON); binder.bind(AccessControlConfig.class).in(Scopes.SINGLETON); binder.bind(TestingAccessControlManager.class).in(Scopes.SINGLETON); binder.bind(TestingGroupProvider.class).in(Scopes.SINGLETON); @@ -337,6 +341,7 @@ private TestingTrinoServer( binder.bind(NodeStateManager.class).in(Scopes.SINGLETON); binder.bind(ProcedureTester.class).in(Scopes.SINGLETON); binder.bind(ExchangeManagerRegistry.class).in(Scopes.SINGLETON); + binder.bind(CacheManagerRegistry.class).in(Scopes.SINGLETON); spanProcessor.ifPresent(processor -> newSetBinder(binder, SpanProcessor.class).addBinding().toInstance(processor)); newSetBinder(binder, SystemSessionPropertiesProvider.class) @@ -428,6 +433,7 @@ private TestingTrinoServer( mBeanServer = injector.getInstance(MBeanServer.class); failureInjector = injector.getInstance(FailureInjector.class); exchangeManagerRegistry = injector.getInstance(ExchangeManagerRegistry.class); + cacheManagerRegistry = injector.getInstance(CacheManagerRegistry.class); spoolingManagerRegistry = injector.getInstance(SpoolingManagerRegistry.class); systemAccessControlConfiguration.ifPresentOrElse( @@ -523,6 +529,11 @@ public void loadExchangeManager(String name, Map properties) exchangeManagerRegistry.loadExchangeManager(name, properties); } + public void loadBlobCacheManager(String name, Map properties) + { + cacheManagerRegistry.loadBlobCacheManager(name, properties); + } + public void loadSpoolingManager(String name, Map properties) { spoolingManagerRegistry.loadSpoolingManager(name, properties); diff --git a/core/trino-main/src/main/java/io/trino/testing/PlanTester.java b/core/trino-main/src/main/java/io/trino/testing/PlanTester.java index 8b801287222a..2f038af66d74 100644 --- a/core/trino-main/src/main/java/io/trino/testing/PlanTester.java +++ b/core/trino-main/src/main/java/io/trino/testing/PlanTester.java @@ -30,6 +30,8 @@ import io.trino.Session; import io.trino.SystemSessionProperties; import io.trino.block.BlockJsonSerde; +import io.trino.cache.CacheManagerConfig; +import io.trino.cache.CacheManagerRegistry; import io.trino.connector.CatalogFactory; import io.trino.connector.CatalogHandle; import io.trino.connector.CatalogServiceProviderModule; @@ -327,6 +329,7 @@ public class PlanTester private final CoordinatorDynamicCatalogManager catalogManager; private final PluginManager pluginManager; private final ExchangeManagerRegistry exchangeManagerRegistry; + private final CacheManagerRegistry cacheManagerRegistry; private final TaskManagerConfig taskManagerConfig; private final OptimizerConfig optimizerConfig; private final StatementAnalyzerFactory statementAnalyzerFactory; @@ -363,7 +366,9 @@ private PlanTester(Session defaultSession, int nodeCountForStats) this.optimizerConfig = new OptimizerConfig(); LazyCatalogFactory catalogFactory = new LazyCatalogFactory(); this.catalogFactory = catalogFactory; - this.catalogManager = new CoordinatorDynamicCatalogManager(new InMemoryCatalogStore(), catalogFactory, directExecutor()); + SecretsResolver secretsResolver = new SecretsResolver(ImmutableMap.of()); + this.cacheManagerRegistry = new CacheManagerRegistry(noop(), noopTracer(), secretsResolver, new CacheManagerConfig()); + this.catalogManager = new CoordinatorDynamicCatalogManager(new InMemoryCatalogStore(), catalogFactory, cacheManagerRegistry, directExecutor()); this.transactionManager = InMemoryTransactionManager.create( new TransactionManagerConfig().setIdleTimeout(new Duration(1, TimeUnit.DAYS)), yieldExecutor, @@ -373,7 +378,6 @@ private PlanTester(Session defaultSession, int nodeCountForStats) TypeRegistry typeRegistry = new TypeRegistry(typeOperators, new FeaturesConfig()); TypeManager typeManager = new InternalTypeManager(typeRegistry); InternalBlockEncodingSerde blockEncodingSerde = new InternalBlockEncodingSerde(TESTING_BLOCK_ENCODING_MANAGER, typeManager); - SecretsResolver secretsResolver = new SecretsResolver(ImmutableMap.of()); this.globalFunctionCatalog = new GlobalFunctionCatalog( () -> getPlannerContext().getMetadata(), @@ -428,7 +432,8 @@ TypeSignature.class, new TypeSignatureDeserializer(), typeManager, nodeSchedulerConfig, optimizerConfig, - secretsResolver)); + secretsResolver, + cacheManagerRegistry)); this.splitManager = new SplitManager(createSplitManagerProvider(catalogManager), tracer, new QueryManagerConfig()); this.pageSourceManager = new PageSourceManager(createPageSourceProviderFactory(catalogManager)); this.pageSinkManager = new PageSinkManager(createPageSinkProvider(catalogManager)); @@ -518,7 +523,8 @@ TypeSignature.class, new TypeSignatureDeserializer(), TESTING_BLOCK_ENCODING_MANAGER, new HandleResolver(), exchangeManagerRegistry, - spoolingManagerRegistry); + spoolingManagerRegistry, + cacheManagerRegistry); catalogManager.registerGlobalSystemConnector(globalSystemConnector); languageFunctionManager.setPlannerContext(plannerContext); @@ -586,6 +592,7 @@ public void close() notificationExecutor.shutdownNow(); yieldExecutor.shutdownNow(); catalogManager.stop(); + cacheManagerRegistry.shutdown(); finalizerService.destroy(); } @@ -665,6 +672,11 @@ public void installPlugin(Plugin plugin) pluginManager.installPlugin(plugin); } + public void loadCacheManager(String name, Map properties) + { + cacheManagerRegistry.loadBlobCacheManager(name, properties); + } + public void addFunctions(FunctionBundle functionBundle) { globalFunctionCatalog.addFunctions(functionBundle); diff --git a/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java index 49f089605485..61706de92489 100644 --- a/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/QueryRunner.java @@ -129,5 +129,7 @@ void injectTaskFailure( void loadSpoolingManager(String name, Map properties); + void loadBlobCacheManager(String name, Map properties); + record MaterializedResultWithPlan(QueryId queryId, Optional queryPlan, MaterializedResult result) {} } diff --git a/core/trino-main/src/main/java/io/trino/testing/StandaloneQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/StandaloneQueryRunner.java index 8a9f1c5ed372..6e105b78806c 100644 --- a/core/trino-main/src/main/java/io/trino/testing/StandaloneQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/StandaloneQueryRunner.java @@ -332,4 +332,10 @@ public void loadSpoolingManager(String name, Map properties) { server.loadSpoolingManager(name, properties); } + + @Override + public void loadBlobCacheManager(String name, Map properties) + { + server.loadBlobCacheManager(name, properties); + } } diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorCacheFactory.java b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorCacheFactory.java new file mode 100644 index 000000000000..967f7517239c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorCacheFactory.java @@ -0,0 +1,50 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.testing; + +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; +import io.trino.spi.cache.CacheLatency; +import io.trino.spi.cache.ConnectorCacheFactory; +import io.trino.spi.cache.PassThroughBlob; + +import java.util.Collection; + +public class TestingConnectorCacheFactory + implements ConnectorCacheFactory +{ + @Override + public BlobCache createBlobCache(CacheLatency latency) + { + return new TestingBlobCache(); + } + + private static class TestingBlobCache + implements BlobCache + { + @Override + public Blob get(CacheKey key, BlobSource source) + { + return new PassThroughBlob(source); + } + + @Override + public void invalidate(CacheKey key) {} + + @Override + public void invalidate(Collection keys) {} + } +} diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java index 45754d6438f6..2853b156b9e0 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java @@ -27,6 +27,7 @@ import io.trino.spi.PageIndexerFactory; import io.trino.spi.PageSorter; import io.trino.spi.VersionEmbedder; +import io.trino.spi.cache.ConnectorCacheFactory; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.MetadataProvider; import io.trino.spi.function.FunctionBundleFactory; @@ -109,4 +110,10 @@ public FunctionBundleFactory getFunctionBundleFactory() { return new InternalFunctionBundleFactory(); } + + @Override + public ConnectorCacheFactory getCacheFactory() + { + return new TestingConnectorCacheFactory(); + } } diff --git a/core/trino-server-core/src/main/provisio/trino-core.xml b/core/trino-server-core/src/main/provisio/trino-core.xml index fe20911f00c0..792080556aa3 100644 --- a/core/trino-server-core/src/main/provisio/trino-core.xml +++ b/core/trino-server-core/src/main/provisio/trino-core.xml @@ -39,6 +39,18 @@ + + + + + + + + + + + + diff --git a/core/trino-spi/src/main/java/io/trino/spi/Plugin.java b/core/trino-spi/src/main/java/io/trino/spi/Plugin.java index 590f06e727c7..5f57be385af4 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/Plugin.java +++ b/core/trino-spi/src/main/java/io/trino/spi/Plugin.java @@ -14,6 +14,7 @@ package io.trino.spi; import io.trino.spi.block.BlockEncoding; +import io.trino.spi.cache.BlobCacheManagerFactory; import io.trino.spi.catalog.CatalogStoreFactory; import io.trino.spi.connector.ConnectorFactory; import io.trino.spi.eventlistener.EventListenerFactory; @@ -121,4 +122,9 @@ default Iterable getSpoolingManagerFactories() { return emptyList(); } + + default Iterable getBlobCacheManagerFactories() + { + return emptyList(); + } } diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/Blob.java b/core/trino-spi/src/main/java/io/trino/spi/cache/Blob.java new file mode 100644 index 000000000000..1a226fce432f --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/Blob.java @@ -0,0 +1,27 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import java.io.Closeable; +import java.io.IOException; + +public interface Blob + extends Closeable +{ + long length() + throws IOException; + + void readFully(long position, byte[] buffer, int offset, int length) + throws IOException; +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCache.java b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCache.java new file mode 100644 index 000000000000..3a48393c3936 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCache.java @@ -0,0 +1,33 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import java.io.IOException; +import java.util.Collection; + +/** + * A filesystem-agnostic blob cache. Callers identify cached bytes with a + * {@link CacheKey} and supply a {@link BlobSource} used to populate the entry + * on miss. The returned {@link BlobSource} reads from the cached entry on + * subsequent calls. + */ +public interface BlobCache +{ + Blob get(CacheKey key, BlobSource source) + throws IOException; + + void invalidate(CacheKey key); + + void invalidate(Collection keys); +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCacheManager.java b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCacheManager.java new file mode 100644 index 000000000000..803170bf02a7 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCacheManager.java @@ -0,0 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import io.trino.spi.catalog.CatalogName; + +public interface BlobCacheManager +{ + BlobCache createBlobCache(CatalogName catalog); + + void drop(CatalogName catalog); + + void shutdown(); +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCacheManagerFactory.java b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCacheManagerFactory.java new file mode 100644 index 000000000000..a526d9795078 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobCacheManagerFactory.java @@ -0,0 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import java.util.Map; + +public interface BlobCacheManagerFactory +{ + String getName(); + + CacheLatency latency(); + + BlobCacheManager create(Map config, CacheManagerContext context); +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/BlobSource.java b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobSource.java new file mode 100644 index 000000000000..f246798d65d5 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/BlobSource.java @@ -0,0 +1,29 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import java.io.IOException; + +/** + * Lazy source of bytes backing a cached blob. Used by {@link BlobCache} to + * populate a cache entry on miss. + */ +public interface BlobSource +{ + long length() + throws IOException; + + void readFully(long position, byte[] buffer, int offset, int length) + throws IOException; +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/CacheKey.java b/core/trino-spi/src/main/java/io/trino/spi/cache/CacheKey.java new file mode 100644 index 000000000000..7320840d4460 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/CacheKey.java @@ -0,0 +1,24 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import static java.util.Objects.requireNonNull; + +public record CacheKey(String key) +{ + public CacheKey + { + requireNonNull(key, "key is null"); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/CacheLatency.java b/core/trino-spi/src/main/java/io/trino/spi/cache/CacheLatency.java new file mode 100644 index 000000000000..dc9b1084626e --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/CacheLatency.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +/** + * Describes latency requirements for caches requested by connectors. + */ +public enum CacheLatency +{ + MEMORY, + DISK, +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/CacheManagerContext.java b/core/trino-spi/src/main/java/io/trino/spi/cache/CacheManagerContext.java new file mode 100644 index 000000000000..ef2864568295 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/CacheManagerContext.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Tracer; + +public interface CacheManagerContext +{ + default OpenTelemetry getOpenTelemetry() + { + throw new UnsupportedOperationException(); + } + + default Tracer getTracer() + { + throw new UnsupportedOperationException(); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileSystemCache.java b/core/trino-spi/src/main/java/io/trino/spi/cache/ConnectorCacheFactory.java similarity index 79% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileSystemCache.java rename to core/trino-spi/src/main/java/io/trino/spi/cache/ConnectorCacheFactory.java index 70d919bf076b..d7cc040c31b7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileSystemCache.java +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/ConnectorCacheFactory.java @@ -11,9 +11,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.fs; +package io.trino.spi.cache; -/** - * Dummy class needed to preserve the legacy JMX object name. - */ -public final class TrinoFileSystemCache {} +public interface ConnectorCacheFactory +{ + BlobCache createBlobCache(CacheLatency latency); +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/cache/PassThroughBlob.java b/core/trino-spi/src/main/java/io/trino/spi/cache/PassThroughBlob.java new file mode 100644 index 000000000000..3191831fe71c --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/cache/PassThroughBlob.java @@ -0,0 +1,54 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.cache; + +import java.io.IOException; + +import static java.util.Objects.requireNonNull; + +public class PassThroughBlob + implements Blob +{ + private final BlobSource source; + + public PassThroughBlob(BlobSource source) + { + this.source = requireNonNull(source, "source is null"); + } + + @Override + public long length() + throws IOException + { + return source.length(); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) + throws IOException + { + source.readFully(position, buffer, offset, length); + } + + @Override + public void close() + { + } + + @Override + public String toString() + { + return source.toString(); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorContext.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorContext.java index ff91d5c5b404..67628c3709bc 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorContext.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorContext.java @@ -21,6 +21,7 @@ import io.trino.spi.PageSorter; import io.trino.spi.Unstable; import io.trino.spi.VersionEmbedder; +import io.trino.spi.cache.ConnectorCacheFactory; import io.trino.spi.function.FunctionBundleFactory; import io.trino.spi.type.TypeManager; @@ -89,4 +90,10 @@ default FunctionBundleFactory getFunctionBundleFactory() { throw new UnsupportedOperationException(); } + + @Unstable + default ConnectorCacheFactory getCacheFactory() + { + throw new UnsupportedOperationException(); + } } diff --git a/core/trino-spi/src/main/java/module-info.java b/core/trino-spi/src/main/java/module-info.java index 270445b97474..246030784ccb 100644 --- a/core/trino-spi/src/main/java/module-info.java +++ b/core/trino-spi/src/main/java/module-info.java @@ -22,6 +22,7 @@ exports io.trino.spi; exports io.trino.spi.block; exports io.trino.spi.catalog; + exports io.trino.spi.cache; exports io.trino.spi.classloader; exports io.trino.spi.connector; exports io.trino.spi.eventlistener; diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java deleted file mode 100644 index 4d063e05e5d9..000000000000 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.alluxio; - -import alluxio.metrics.MetricsConfig; -import alluxio.metrics.MetricsSystem; -import com.google.inject.Binder; -import com.google.inject.Provider; -import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.filesystem.cache.CachingHostAddressProvider; -import io.trino.filesystem.cache.ConsistentHashingHostAddressProvider; -import io.trino.filesystem.cache.ConsistentHashingHostAddressProviderConfig; -import io.trino.filesystem.cache.TrinoFileSystemCache; -import io.trino.spi.catalog.CatalogName; - -import java.util.Properties; - -import static com.google.inject.Scopes.SINGLETON; -import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; -import static io.airlift.configuration.ConfigBinder.configBinder; -import static org.weakref.jmx.guice.ExportBinder.newExporter; - -public class AlluxioFileSystemCacheModule - extends AbstractConfigurationAwareModule -{ - private final boolean isCoordinator; - - public AlluxioFileSystemCacheModule(boolean isCoordinator) - { - this.isCoordinator = isCoordinator; - } - - @Override - protected void setup(Binder binder) - { - configBinder(binder).bindConfig(AlluxioFileSystemCacheConfig.class); - configBinder(binder).bindConfig(ConsistentHashingHostAddressProviderConfig.class); - binder.bind(AlluxioCacheStats.class).in(SINGLETON); - Provider catalogName = binder.getProvider(CatalogName.class); - newExporter(binder).export(AlluxioCacheStats.class) - .as(generator -> generator.generatedNameOf(AlluxioCacheStats.class, catalogName.get().toString())); - - if (isCoordinator) { - newOptionalBinder(binder, CachingHostAddressProvider.class).setBinding().to(ConsistentHashingHostAddressProvider.class).in(SINGLETON); - } - binder.bind(TrinoFileSystemCache.class).to(AlluxioFileSystemCache.class).in(SINGLETON); - - Properties metricProps = new Properties(); - metricProps.put("sink.jmx.class", "alluxio.metrics.sink.JmxSink"); - metricProps.put("sink.jmx.domain", "org.alluxio"); - MetricsSystem.startSinksFromConfig(new MetricsConfig(metricProps)); - } -} diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInput.java b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInput.java deleted file mode 100644 index cbda79126f96..000000000000 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInput.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.alluxio; - -import alluxio.client.file.URIStatus; -import alluxio.client.file.cache.CacheManager; -import alluxio.conf.AlluxioConfiguration; -import com.google.common.collect.ImmutableMap; -import io.opentelemetry.api.trace.Tracer; -import io.trino.filesystem.TrinoInput; -import io.trino.filesystem.TrinoInputFile; -import io.trino.plugin.base.metrics.LongCount; -import io.trino.spi.metrics.Metrics; - -import java.io.EOFException; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; - -import static java.lang.Math.min; -import static java.util.Objects.checkFromIndexSize; -import static java.util.Objects.requireNonNull; - -public class AlluxioInput - implements TrinoInput -{ - private final TrinoInputFile inputFile; - private final long fileLength; - private final AlluxioCacheStats statistics; - private final AlluxioInputHelper helper; - private final AtomicLong externalReadBytes; - - private TrinoInput input; - private boolean closed; - - public AlluxioInput( - Tracer tracer, - TrinoInputFile inputFile, - String cacheKey, - URIStatus status, - CacheManager cacheManager, - AlluxioConfiguration configuration, - AlluxioCacheStats statistics) - { - this.inputFile = requireNonNull(inputFile, "inputFile is null"); - this.fileLength = requireNonNull(status, "status is null").getLength(); - this.statistics = requireNonNull(statistics, "statistics is null"); - this.helper = new AlluxioInputHelper(tracer, inputFile.location(), cacheKey, status, cacheManager, configuration, statistics); - this.externalReadBytes = new AtomicLong(); - } - - @Override - public void readFully(long position, byte[] buffer, int offset, int length) - throws IOException - { - ensureOpen(); - checkFromIndexSize(offset, length, buffer.length); - if (position < 0) { - throw new IOException("Negative seek offset"); - } - if (length == 0) { - return; - } - - int bytesRead = helper.doCacheRead(position, buffer, offset, length); - if (length > bytesRead && position + bytesRead == fileLength) { - throw new EOFException("Read %s of %s requested bytes: %s".formatted(bytesRead, length, inputFile.location())); - } - doExternalRead(position + bytesRead, buffer, offset + bytesRead, length - bytesRead); - } - - private int doExternalRead(long position, byte[] buffer, int offset, int length) - throws IOException - { - if (length == 0) { - return 0; - } - - AlluxioInputHelper.PageAlignedRead aligned = helper.alignRead(position, length); - byte[] readBuffer = new byte[aligned.length()]; - getInput().readFully(aligned.pageStart(), readBuffer, 0, readBuffer.length); - helper.putCache(aligned.pageStart(), aligned.pageEnd(), readBuffer, aligned.length()); - System.arraycopy(readBuffer, aligned.pageOffset(), buffer, offset, length); - statistics.recordExternalRead(readBuffer.length); - externalReadBytes.addAndGet(readBuffer.length); - return length; - } - - private TrinoInput getInput() - throws IOException - { - if (input == null) { - input = inputFile.newInput(); - } - return input; - } - - @Override - public int readTail(byte[] buffer, int bufferOffset, int bufferLength) - throws IOException - { - ensureOpen(); - checkFromIndexSize(bufferOffset, bufferLength, buffer.length); - - int readSize = (int) min(fileLength, bufferLength); - readFully(fileLength - readSize, buffer, bufferOffset, readSize); - return readSize; - } - - private void ensureOpen() - throws IOException - { - if (closed) { - throw new IOException("Stream closed: " + inputFile.location()); - } - } - - @Override - public Metrics getMetrics() - { - return new Metrics(ImmutableMap.of( - "bytesReadFromCache", new LongCount(helper.getCacheReadBytes()), - "bytesReadExternally", new LongCount(externalReadBytes.get()))); - } - - @Override - public void close() - throws IOException - { - closed = true; - if (input != null) { - input.close(); - } - } -} diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInputStream.java b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInputStream.java deleted file mode 100644 index a2d12ee229a2..000000000000 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInputStream.java +++ /dev/null @@ -1,236 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.alluxio; - -import alluxio.client.file.URIStatus; -import alluxio.client.file.cache.CacheManager; -import alluxio.conf.AlluxioConfiguration; -import com.google.common.primitives.Longs; -import io.opentelemetry.api.trace.Span; -import io.opentelemetry.api.trace.Tracer; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.TrinoInputStream; - -import java.io.EOFException; -import java.io.IOException; - -import static com.google.common.base.Verify.verify; -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_LOCATION; -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_READ_POSITION; -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_READ_SIZE; -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_KEY; -import static io.trino.filesystem.tracing.Tracing.withTracing; -import static java.lang.Integer.max; -import static java.lang.Math.addExact; -import static java.lang.Math.min; -import static java.lang.Math.toIntExact; -import static java.lang.String.format; -import static java.util.Objects.checkFromIndexSize; -import static java.util.Objects.requireNonNull; - -public class AlluxioInputStream - extends TrinoInputStream -{ - private final TrinoInputFile inputFile; - private final long fileLength; - private final Location location; - private final AlluxioCacheStats statistics; - private final String key; - private final AlluxioInputHelper helper; - private final Tracer tracer; - private TrinoInputStream externalStream; - private long position; - private boolean closed; - - public AlluxioInputStream(Tracer tracer, TrinoInputFile inputFile, String key, URIStatus status, CacheManager cacheManager, AlluxioConfiguration configuration, AlluxioCacheStats statistics) - { - this.tracer = requireNonNull(tracer, "tracer is null"); - this.inputFile = requireNonNull(inputFile, "inputFile is null"); - this.fileLength = requireNonNull(status, "status is null").getLength(); - this.location = inputFile.location(); - this.statistics = requireNonNull(statistics, "statistics is null"); - this.key = requireNonNull(key, "key is null"); - this.helper = new AlluxioInputHelper(tracer, inputFile.location(), key, status, cacheManager, configuration, statistics); - } - - @Override - public int available() - throws IOException - { - // Not needed per contract, but complies with AbstractTestTrinoFileSystem expectations easier. - // It's easer to just check "is open?" in available() than refactor that test. - ensureOpen(); - return super.available(); - } - - @Override - public long getPosition() - { - return position; - } - - private void ensureOpen() - throws IOException - { - if (closed) { - throw new IOException("Output stream closed: " + location); - } - } - - @Override - public int read() - throws IOException - { - ensureOpen(); - - byte[] bytes = new byte[1]; - int n = read(bytes, 0, 1); - if (n == 1) { - // Converts the byte to an unsigned byte, an integer in the range 0 to 255 - return bytes[0] & 0xff; - } - if (n == -1) { - return -1; - } - throw new IOException(format("%d bytes read", n)); - } - - @Override - public int read(byte[] bytes, int offset, int length) - throws IOException - { - ensureOpen(); - - checkFromIndexSize(offset, length, bytes.length); - if (length == 0) { - return 0; - } - if (position >= fileLength) { - return -1; - } - int bytesRead = doRead(bytes, offset, toIntExact(min(fileLength - position, length))); - position += bytesRead; - return bytesRead; - } - - private int doRead(byte[] bytes, int offset, int length) - throws IOException - { - int bytesRead = helper.doCacheRead(position, bytes, offset, length); - return addExact(bytesRead, doExternalRead0(position + bytesRead, bytes, offset + bytesRead, length - bytesRead)); - } - - private int doExternalRead0(long readPosition, byte[] buffer, int offset, int length) - throws IOException - { - if (length == 0) { - return 0; - } - - Span span = tracer.spanBuilder("Alluxio.readExternalStream") - .setAttribute(CACHE_KEY, key) - .setAttribute(CACHE_FILE_LOCATION, inputFile.location().toString()) - .setAttribute(CACHE_FILE_READ_SIZE, (long) length) - .setAttribute(CACHE_FILE_READ_POSITION, readPosition) - .startSpan(); - - return withTracing(span, () -> doExternalReadInternal(readPosition, buffer, offset, length)); - } - - private int doExternalReadInternal(long readPosition, byte[] buffer, int offset, int length) - throws IOException - { - verify(length > 0, "zero-length or negative read"); - AlluxioInputHelper.PageAlignedRead aligned = helper.alignRead(readPosition, length); - if (externalStream == null) { - externalStream = inputFile.newStream(); - } - externalStream.seek(aligned.pageStart()); - byte[] readBuffer = new byte[aligned.length()]; - int externalBytesRead = externalStream.readNBytes(readBuffer, 0, aligned.length()); - if (externalBytesRead < 0) { - throw new IOException("Unexpected end of stream"); - } - verify(aligned.length() == externalBytesRead, "invalid number of external bytes read"); - helper.putCache(aligned.pageStart(), aligned.pageEnd(), readBuffer, externalBytesRead); - int bytesToCopy = min(length, max(externalBytesRead - aligned.pageOffset(), 0)); - System.arraycopy(readBuffer, aligned.pageOffset(), buffer, offset, bytesToCopy); - statistics.recordExternalRead(externalBytesRead); - return bytesToCopy; - } - - @Override - public long skip(long n) - throws IOException - { - ensureOpen(); - - n = Longs.constrainToRange(n, 0, fileLength - position); - position += n; - return n; - } - - @Override - public void skipNBytes(long n) - throws IOException - { - ensureOpen(); - - if (n <= 0) { - return; - } - - long position; - try { - position = addExact(this.position, n); - } - catch (ArithmeticException e) { - throw new EOFException("Unable to skip %s bytes (position=%s, fileSize=%s): %s".formatted(n, this.position, fileLength, location)); - } - if (position > fileLength) { - throw new EOFException("Unable to skip %s bytes (position=%s, fileSize=%s): %s".formatted(n, this.position, fileLength, location)); - } - this.position = position; - } - - @Override - public void seek(long position) - throws IOException - { - ensureOpen(); - - if (position < 0) { - throw new IOException("Negative seek offset"); - } - if (position > fileLength) { - throw new IOException("Cannot seek to %s. File size is %s: %s".formatted(position, fileLength, location)); - } - - this.position = position; - } - - @Override - public void close() - throws IOException - { - if (!closed) { - closed = true; - if (externalStream != null) { - externalStream.close(); - externalStream = null; - } - } - } -} diff --git a/lib/trino-filesystem-manager/pom.xml b/lib/trino-filesystem-manager/pom.xml index 0d9183385c34..99f32b08e305 100644 --- a/lib/trino-filesystem-manager/pom.xml +++ b/lib/trino-filesystem-manager/pom.xml @@ -54,11 +54,6 @@ trino-filesystem-azure - - io.trino - trino-filesystem-cache-alluxio - - io.trino trino-filesystem-gcs diff --git a/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java b/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java index 8183e434f61c..d1632edfdb60 100644 --- a/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java +++ b/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java @@ -18,12 +18,12 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.Singleton; +import com.google.inject.multibindings.OptionalBinder; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.configuration.ConfigPropertyMetadata; import io.opentelemetry.api.trace.Tracer; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; -import io.trino.filesystem.alluxio.AlluxioFileSystemCacheModule; import io.trino.filesystem.alluxio.AlluxioFileSystemFactory; import io.trino.filesystem.alluxio.AlluxioFileSystemModule; import io.trino.filesystem.azure.AzureFileSystemFactory; @@ -33,18 +33,17 @@ import io.trino.filesystem.cache.CachingHostAddressProvider; import io.trino.filesystem.cache.DefaultCacheKeyProvider; import io.trino.filesystem.cache.DefaultCachingHostAddressProvider; -import io.trino.filesystem.cache.TrinoFileSystemCache; import io.trino.filesystem.gcs.GcsFileSystemFactory; import io.trino.filesystem.gcs.GcsFileSystemModule; import io.trino.filesystem.local.LocalFileSystemConfig; import io.trino.filesystem.local.LocalFileSystemFactory; -import io.trino.filesystem.memory.MemoryFileSystemCache; -import io.trino.filesystem.memory.MemoryFileSystemCacheModule; import io.trino.filesystem.s3.FileSystemS3; import io.trino.filesystem.s3.S3FileSystemModule; import io.trino.filesystem.switching.SwitchingFileSystemFactory; import io.trino.filesystem.tracing.TracingFileSystemFactory; import io.trino.filesystem.tracking.TrackingFileSystemFactory; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.CacheLatency; import io.trino.spi.connector.ConnectorContext; import java.util.Map; @@ -124,14 +123,12 @@ protected void setup(Binder binder) newOptionalBinder(binder, CachingHostAddressProvider.class).setDefault().to(DefaultCachingHostAddressProvider.class).in(Scopes.SINGLETON); newOptionalBinder(binder, CacheKeyProvider.class).setDefault().to(DefaultCacheKeyProvider.class).in(Scopes.SINGLETON); - newOptionalBinder(binder, TrinoFileSystemCache.class); - newOptionalBinder(binder, MemoryFileSystemCache.class); - + OptionalBinder blobCacheBinder = newOptionalBinder(binder, BlobCache.class); if (config.isCacheEnabled()) { - install(new AlluxioFileSystemCacheModule(isCoordinator)); + blobCacheBinder.setBinding().toInstance(context.getCacheFactory().createBlobCache(CacheLatency.DISK)); } - if (coordinatorFileCaching) { - install(new MemoryFileSystemCacheModule(isCoordinator)); + else if (coordinatorFileCaching && isCoordinator) { + blobCacheBinder.setBinding().toInstance(context.getCacheFactory().createBlobCache(CacheLatency.MEMORY)); } } @@ -141,8 +138,7 @@ static TrinoFileSystemFactory createFileSystemFactory( FileSystemConfig config, Optional hdfsFileSystemLoader, Map factories, - Optional fileSystemCache, - Optional memoryFileSystemCache, + Optional blobCache, Optional keyProvider, Tracer tracer) { @@ -160,12 +156,8 @@ static TrinoFileSystemFactory createFileSystemFactory( delegate = new TrackingFileSystemFactory(delegate); } - if (fileSystemCache.isPresent()) { - return new CacheFileSystemFactory(tracer, delegate, fileSystemCache.orElseThrow(), keyProvider.orElseThrow()); - } - // use MemoryFileSystemCache only when no other TrinoFileSystemCache is configured - if (memoryFileSystemCache.isPresent()) { - return new CacheFileSystemFactory(tracer, delegate, memoryFileSystemCache.orElseThrow(), keyProvider.orElseThrow()); + if (blobCache.isPresent()) { + return new CacheFileSystemFactory(tracer, delegate, blobCache.orElseThrow(), keyProvider.orElseThrow()); } return delegate; } diff --git a/lib/trino-filesystem/pom.xml b/lib/trino-filesystem/pom.xml index ac7dcb5f4bc4..68507cd18125 100644 --- a/lib/trino-filesystem/pom.xml +++ b/lib/trino-filesystem/pom.xml @@ -70,21 +70,11 @@ opentelemetry-context - - io.trino - trino-cache - - io.trino trino-memory-context - - io.trino - trino-plugin-toolkit - - io.trino trino-spi @@ -95,16 +85,6 @@ jakarta.annotation-api - - jakarta.validation - jakarta.validation-api - - - - org.weakref - jmxutils - - org.jetbrains annotations diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/BlobTrinoInput.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/BlobTrinoInput.java new file mode 100644 index 000000000000..95e3ce366f05 --- /dev/null +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/BlobTrinoInput.java @@ -0,0 +1,75 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.filesystem.cache; + +import io.trino.filesystem.TrinoInput; +import io.trino.spi.cache.Blob; + +import java.io.IOException; + +import static java.lang.Math.min; +import static java.util.Objects.checkFromIndexSize; +import static java.util.Objects.requireNonNull; + +final class BlobTrinoInput + implements TrinoInput +{ + private final Blob blob; + private boolean closed; + + BlobTrinoInput(Blob blob) + { + this.blob = requireNonNull(blob, "blob is null"); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) + throws IOException + { + ensureOpen(); + blob.readFully(position, buffer, offset, length); + } + + @Override + public int readTail(byte[] buffer, int offset, int length) + throws IOException + { + ensureOpen(); + checkFromIndexSize(offset, length, buffer.length); + long size = blob.length(); + int readSize = (int) min(size, length); + readFully(size - readSize, buffer, offset, readSize); + return readSize; + } + + @Override + public void close() + { + closed = true; + } + + private void ensureOpen() + throws IOException + { + if (closed) { + throw new IOException("Input closed: " + blob); + } + } + + @Override + public String toString() + { + return blob.toString(); + } +} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/BlobTrinoInputStream.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/BlobTrinoInputStream.java new file mode 100644 index 000000000000..9c21dc1660a3 --- /dev/null +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/BlobTrinoInputStream.java @@ -0,0 +1,138 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.filesystem.cache; + +import io.trino.filesystem.TrinoInputStream; +import io.trino.spi.cache.Blob; + +import java.io.IOException; + +import static java.lang.Math.min; +import static java.lang.Math.toIntExact; +import static java.util.Objects.checkFromIndexSize; +import static java.util.Objects.requireNonNull; + +final class BlobTrinoInputStream + extends TrinoInputStream +{ + private final Blob blob; + private final long length; + private long position; + private boolean closed; + + BlobTrinoInputStream(Blob blob) + throws IOException + { + this.blob = requireNonNull(blob, "blob is null"); + this.length = blob.length(); + } + + @Override + public long getPosition() + { + return position; + } + + @Override + public void seek(long newPosition) + throws IOException + { + ensureOpen(); + if (newPosition < 0) { + throw new IOException("Negative seek offset"); + } + if (newPosition > length) { + throw new IOException("Cannot seek to %s. Blob size is %s: %s".formatted(newPosition, length, blob)); + } + this.position = newPosition; + } + + @Override + public int read() + throws IOException + { + ensureOpen(); + if (position >= length) { + return -1; + } + byte[] single = new byte[1]; + blob.readFully(position, single, 0, 1); + position++; + return single[0] & 0xFF; + } + + @Override + public int read(byte[] buffer, int offset, int len) + throws IOException + { + ensureOpen(); + checkFromIndexSize(offset, len, buffer.length); + if (len == 0) { + return 0; + } + if (position >= length) { + return -1; + } + int toRead = toIntExact(min(len, length - position)); + blob.readFully(position, buffer, offset, toRead); + position += toRead; + return toRead; + } + + @Override + public long skip(long n) + throws IOException + { + ensureOpen(); + if (n <= 0) { + return 0; + } + long skipped = min(n, length - position); + position += skipped; + return skipped; + } + + @Override + public int available() + throws IOException + { + ensureOpen(); + long remaining = length - position; + return (remaining > Integer.MAX_VALUE) ? Integer.MAX_VALUE : (int) remaining; + } + + @Override + public void close() + throws IOException + { + if (closed) { + return; + } + closed = true; + try { + blob.close(); + } + catch (Exception e) { + throw new IOException("Could not close cached blob", e); + } + } + + private void ensureOpen() + throws IOException + { + if (closed) { + throw new IOException("Cached blob is closed: " + blob); + } + } +} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystem.java index cdf338825d48..27f62940574a 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystem.java @@ -13,14 +13,15 @@ */ package io.trino.filesystem.cache; +import io.airlift.log.Logger; import io.trino.filesystem.FileIterator; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; +import io.trino.spi.cache.BlobCache; import java.io.IOException; -import java.io.UncheckedIOException; import java.time.Instant; import java.util.Collection; import java.util.Optional; @@ -32,11 +33,13 @@ public final class CacheFileSystem implements TrinoFileSystem { + private static final Logger log = Logger.get(CacheFileSystem.class); + private final TrinoFileSystem delegate; - private final TrinoFileSystemCache cache; + private final BlobCache cache; private final CacheKeyProvider keyProvider; - public CacheFileSystem(TrinoFileSystem delegate, TrinoFileSystemCache cache, CacheKeyProvider keyProvider) + public CacheFileSystem(TrinoFileSystem delegate, BlobCache cache, CacheKeyProvider keyProvider) { this.delegate = requireNonNull(delegate, "delegate is null"); this.cache = requireNonNull(cache, "cache is null"); @@ -64,39 +67,33 @@ public TrinoInputFile newInputFile(Location location, long length, Instant lastM @Override public TrinoOutputFile newOutputFile(Location location) { - TrinoOutputFile output = delegate.newOutputFile(location); - try { - cache.expire(location); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - return output; + invalidate(location); + return delegate.newOutputFile(location); } @Override public void deleteFile(Location location) throws IOException { + invalidate(location); delegate.deleteFile(location); - cache.expire(location); } @Override public void deleteDirectory(Location location) throws IOException { + invalidateDirectoryEntries(location); delegate.deleteDirectory(location); - cache.expire(location); } @Override public void renameFile(Location source, Location target) throws IOException { + invalidate(source); + invalidate(target); delegate.renameFile(source, target); - cache.expire(source); - cache.expire(target); } @Override @@ -124,6 +121,8 @@ public void createDirectory(Location location) public void renameDirectory(Location source, Location target) throws IOException { + invalidateDirectoryEntries(source); + invalidateDirectoryEntries(target); delegate.renameDirectory(source, target); } @@ -145,7 +144,31 @@ public Optional createTemporaryDirectory(Location targetPath, String t public void deleteFiles(Collection locations) throws IOException { + locations.forEach(this::invalidate); delegate.deleteFiles(locations); - cache.expire(locations); + } + + private void invalidate(Location location) + { + try { + keyProvider.getCacheKey(delegate.newInputFile(location)) + .ifPresent(cache::invalidate); + } + catch (IOException e) { + log.warn(e, "Failed to invalidate cache entry for %s", location); + } + } + + private void invalidateDirectoryEntries(Location location) + { + try { + FileIterator iterator = delegate.listFiles(location); + while (iterator.hasNext()) { + invalidate(iterator.next().location()); + } + } + catch (IOException e) { + log.warn(e, "Failed to invalidate cache entries under %s", location); + } } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystemFactory.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystemFactory.java index 3c02e0a3c4f3..646ca4a87b2a 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystemFactory.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheFileSystemFactory.java @@ -16,7 +16,8 @@ import io.opentelemetry.api.trace.Tracer; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; -import io.trino.filesystem.tracing.TracingFileSystemCache; +import io.trino.filesystem.tracing.TracingBlobCache; +import io.trino.spi.cache.BlobCache; import io.trino.spi.security.ConnectorIdentity; import static java.util.Objects.requireNonNull; @@ -26,10 +27,10 @@ public final class CacheFileSystemFactory { private final Tracer tracer; private final TrinoFileSystemFactory delegate; - private final TrinoFileSystemCache cache; + private final BlobCache cache; private final CacheKeyProvider keyProvider; - public CacheFileSystemFactory(Tracer tracer, TrinoFileSystemFactory delegate, TrinoFileSystemCache cache, CacheKeyProvider keyProvider) + public CacheFileSystemFactory(Tracer tracer, TrinoFileSystemFactory delegate, BlobCache cache, CacheKeyProvider keyProvider) { this.tracer = requireNonNull(tracer, "tracer is null"); this.delegate = requireNonNull(delegate, "delegate is null"); @@ -40,6 +41,6 @@ public CacheFileSystemFactory(Tracer tracer, TrinoFileSystemFactory delegate, Tr @Override public TrinoFileSystem create(ConnectorIdentity identity) { - return new CacheFileSystem(delegate.create(identity), new TracingFileSystemCache(tracer, cache), keyProvider); + return new CacheFileSystem(delegate.create(identity), new TracingBlobCache(tracer, cache), keyProvider); } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheInputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheInputFile.java index 4db7ed718f69..d0bc59d9690b 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheInputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheInputFile.java @@ -17,6 +17,9 @@ import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; import java.io.IOException; import java.time.Instant; @@ -30,12 +33,12 @@ public final class CacheInputFile implements TrinoInputFile { private final TrinoInputFile delegate; - private final TrinoFileSystemCache cache; + private final BlobCache cache; private final CacheKeyProvider keyProvider; private OptionalLong length; private Optional lastModified; - public CacheInputFile(TrinoInputFile delegate, TrinoFileSystemCache cache, CacheKeyProvider keyProvider, OptionalLong length, Optional lastModified) + public CacheInputFile(TrinoInputFile delegate, BlobCache cache, CacheKeyProvider keyProvider, OptionalLong length, Optional lastModified) { this.delegate = requireNonNull(delegate, "delegate is null"); this.cache = requireNonNull(cache, "cache is null"); @@ -48,9 +51,9 @@ public CacheInputFile(TrinoInputFile delegate, TrinoFileSystemCache cache, Cache public TrinoInput newInput() throws IOException { - Optional key = keyProvider.getCacheKey(delegate); + Optional key = keyProvider.getCacheKey(delegate); if (key.isPresent()) { - return cache.cacheInput(delegate, key.orElseThrow()); + return new BlobTrinoInput(cache.get(key.orElseThrow(), source())); } return delegate.newInput(); } @@ -59,9 +62,9 @@ public TrinoInput newInput() public TrinoInputStream newStream() throws IOException { - Optional key = keyProvider.getCacheKey(delegate); + Optional key = keyProvider.getCacheKey(delegate); if (key.isPresent()) { - return cache.cacheStream(delegate, key.orElseThrow()); + return new BlobTrinoInputStream(cache.get(key.orElseThrow(), source())); } return delegate.newStream(); } @@ -71,9 +74,9 @@ public long length() throws IOException { if (length.isEmpty()) { - Optional key = keyProvider.getCacheKey(delegate); + Optional key = keyProvider.getCacheKey(delegate); if (key.isPresent()) { - length = OptionalLong.of(cache.cacheLength(delegate, key.orElseThrow())); + length = OptionalLong.of(cache.get(key.orElseThrow(), source()).length()); } else { length = OptionalLong.of(delegate.length()); @@ -105,6 +108,11 @@ public Location location() return delegate.location(); } + private BlobSource source() + { + return new TrinoInputFileBlobSource(delegate); + } + @Override public String toString() { diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheKeyProvider.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheKeyProvider.java index f557e938a3ea..2c8712e67f58 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheKeyProvider.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheKeyProvider.java @@ -14,6 +14,7 @@ package io.trino.filesystem.cache; import io.trino.filesystem.TrinoInputFile; +import io.trino.spi.cache.CacheKey; import java.io.IOException; import java.util.Optional; @@ -23,6 +24,6 @@ public interface CacheKeyProvider /** * Get the cache key of a TrinoInputFile. Returns Optional.empty() if the file is not cacheable. */ - Optional getCacheKey(TrinoInputFile inputFile) + Optional getCacheKey(TrinoInputFile inputFile) throws IOException; } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCacheKeyProvider.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCacheKeyProvider.java index e297691544a2..ab9cccbecc4f 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCacheKeyProvider.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCacheKeyProvider.java @@ -14,6 +14,7 @@ package io.trino.filesystem.cache; import io.trino.filesystem.TrinoInputFile; +import io.trino.spi.cache.CacheKey; import java.io.IOException; import java.util.Optional; @@ -22,9 +23,9 @@ public final class DefaultCacheKeyProvider implements CacheKeyProvider { @Override - public Optional getCacheKey(TrinoInputFile inputFile) + public Optional getCacheKey(TrinoInputFile inputFile) throws IOException { - return Optional.of(inputFile.location().path() + "#" + inputFile.lastModified() + "#" + inputFile.length()); + return Optional.of(new CacheKey(inputFile.location() + "#" + inputFile.lastModified() + "#" + inputFile.length())); } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/TrinoFileSystemCache.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/TrinoFileSystemCache.java deleted file mode 100644 index 6f4b3e588788..000000000000 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/TrinoFileSystemCache.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.cache; - -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoInput; -import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.TrinoInputStream; - -import java.io.IOException; -import java.util.Collection; - -public interface TrinoFileSystemCache -{ - /** - * Get the TrinoInput of the TrinoInputFile, potentially using or updating the data cached at key. - */ - TrinoInput cacheInput(TrinoInputFile delegate, String key) - throws IOException; - - /** - * Get the TrinoInputStream of the TrinoInputFile, potentially using or updating the data cached at key. - */ - TrinoInputStream cacheStream(TrinoInputFile delegate, String key) - throws IOException; - - /** - * Get the length of the TrinoInputFile, potentially using or updating the data cached at key. - */ - long cacheLength(TrinoInputFile delegate, String key) - throws IOException; - - /** - * Give a hint to the cache that the cache entry for location should be expired. - */ - void expire(Location location) - throws IOException; - - /** - * Give a hint to the cache that the cache entry for locations should be expired. - */ - void expire(Collection locations) - throws IOException; -} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/TrinoInputFileBlobSource.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/TrinoInputFileBlobSource.java new file mode 100644 index 000000000000..4eaa8f2376d5 --- /dev/null +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/TrinoInputFileBlobSource.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.filesystem.cache; + +import io.trino.filesystem.TrinoInput; +import io.trino.filesystem.TrinoInputFile; +import io.trino.spi.cache.BlobSource; + +import java.io.IOException; + +import static java.util.Objects.requireNonNull; + +public final class TrinoInputFileBlobSource + implements BlobSource +{ + private final TrinoInputFile delegate; + + public TrinoInputFileBlobSource(TrinoInputFile delegate) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + } + + @Override + public long length() + throws IOException + { + return delegate.length(); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) + throws IOException + { + try (TrinoInput input = delegate.newInput()) { + input.readFully(position, buffer, offset, length); + } + } + + @Override + public String toString() + { + return delegate.location().toString(); + } +} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCache.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCache.java deleted file mode 100644 index bb03c1569071..000000000000 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCache.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.memory; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.cache.Cache; -import com.google.common.cache.Weigher; -import com.google.inject.Inject; -import io.airlift.slice.Slice; -import io.airlift.units.DataSize; -import io.airlift.units.Duration; -import io.trino.cache.EvictableCacheBuilder; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoInput; -import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.TrinoInputStream; -import io.trino.filesystem.cache.TrinoFileSystemCache; -import org.weakref.jmx.Managed; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.file.FileAlreadyExistsException; -import java.nio.file.NoSuchFileException; -import java.util.Collection; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.airlift.slice.SizeOf.estimatedSizeOf; -import static io.airlift.slice.SizeOf.sizeOf; -import static io.airlift.units.DataSize.Unit.GIGABYTE; -import static java.lang.Math.toIntExact; - -public final class MemoryFileSystemCache - implements TrinoFileSystemCache -{ - private final Cache> cache; - private final int maxContentLengthBytes; - private final AtomicLong largeFileSkippedCount = new AtomicLong(); - - @Inject - public MemoryFileSystemCache(MemoryFileSystemCacheConfig config) - { - this(config.getCacheTtl(), config.getMaxSize(), config.getMaxContentLength()); - } - - private MemoryFileSystemCache(Duration expireAfterWrite, DataSize maxSize, DataSize maxContentLength) - { - checkArgument(maxContentLength.compareTo(DataSize.of(1, GIGABYTE)) <= 0, "maxContentLength must be less than or equal to 1GB"); - this.cache = EvictableCacheBuilder.newBuilder() - .maximumWeight(maxSize.toBytes()) - .weigher((Weigher>) (key, value) -> toIntExact(estimatedSizeOf(key) + sizeOf(value, Slice::getRetainedSize))) - .expireAfterWrite(expireAfterWrite.toMillis(), TimeUnit.MILLISECONDS) - .shareNothingWhenDisabled() - .recordStats() - .build(); - this.maxContentLengthBytes = toIntExact(maxContentLength.toBytes()); - } - - @Override - public TrinoInput cacheInput(TrinoInputFile delegate, String key) - throws IOException - { - Optional cachedEntry = getOrLoadFromCache(key, delegate); - if (cachedEntry.isEmpty()) { - largeFileSkippedCount.incrementAndGet(); - return delegate.newInput(); - } - - return new MemoryInput(delegate.location(), cachedEntry.get()); - } - - @Override - public TrinoInputStream cacheStream(TrinoInputFile delegate, String key) - throws IOException - { - Optional cachedEntry = getOrLoadFromCache(key, delegate); - if (cachedEntry.isEmpty()) { - largeFileSkippedCount.incrementAndGet(); - return delegate.newStream(); - } - - return new MemoryInputStream(delegate.location(), cachedEntry.get()); - } - - @Override - public long cacheLength(TrinoInputFile delegate, String key) - throws IOException - { - Optional cachedEntry = getOrLoadFromCache(key, delegate); - if (cachedEntry.isEmpty()) { - largeFileSkippedCount.incrementAndGet(); - return delegate.length(); - } - - return cachedEntry.get().length(); - } - - @Override - public void expire(Location location) - throws IOException - { - List expired = cache.asMap().keySet().stream() - .filter(key -> key.startsWith(location.path())) - .collect(toImmutableList()); - cache.invalidateAll(expired); - } - - @Override - public void expire(Collection locations) - throws IOException - { - List expired = cache.asMap().keySet().stream() - .filter(key -> locations.stream().map(Location::path).anyMatch(key::startsWith)) - .collect(toImmutableList()); - cache.invalidateAll(expired); - } - - @Managed - public void flushCache() - { - cache.invalidateAll(); - } - - @Managed - public long getHitCount() - { - return cache.stats().hitCount(); - } - - @Managed - public long getRequestCount() - { - return cache.stats().requestCount(); - } - - @Managed - public long getLargeFileSkippedCount() - { - return largeFileSkippedCount.get(); - } - - @VisibleForTesting - boolean isCached(String key) - { - Optional cachedEntry = cache.getIfPresent(key); - return cachedEntry != null && cachedEntry.isPresent(); - } - - private Optional getOrLoadFromCache(String key, TrinoInputFile delegate) - throws IOException - { - try { - return cache.get(key, () -> load(delegate)); - } - catch (ExecutionException e) { - throw handleException(delegate.location(), e.getCause()); - } - } - - private Optional load(TrinoInputFile delegate) - throws IOException - { - long fileSize = delegate.length(); - if (fileSize > maxContentLengthBytes) { - return Optional.empty(); - } - try (TrinoInput trinoInput = delegate.newInput()) { - return Optional.of(trinoInput.readTail(toIntExact(fileSize))); - } - } - - private static IOException handleException(Location location, Throwable cause) - throws IOException - { - if (cause instanceof FileNotFoundException || cause instanceof NoSuchFileException) { - throw withCause(new FileNotFoundException(location.toString()), cause); - } - if (cause instanceof FileAlreadyExistsException) { - throw withCause(new FileAlreadyExistsException(location.toString()), cause); - } - throw new IOException(cause.getMessage() + ": " + location, cause); - } - - private static T withCause(T throwable, Throwable cause) - { - throwable.initCause(cause); - return throwable; - } -} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCacheModule.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCacheModule.java deleted file mode 100644 index 1b4bd9470dd9..000000000000 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCacheModule.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.memory; - -import com.google.inject.Binder; -import com.google.inject.Provider; -import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.spi.catalog.CatalogName; - -import static com.google.inject.Scopes.SINGLETON; -import static io.airlift.configuration.ConfigBinder.configBinder; -import static org.weakref.jmx.guice.ExportBinder.newExporter; - -public class MemoryFileSystemCacheModule - extends AbstractConfigurationAwareModule -{ - private final boolean isCoordinator; - - public MemoryFileSystemCacheModule(boolean isCoordinator) - { - this.isCoordinator = isCoordinator; - } - - @Override - protected void setup(Binder binder) - { - configBinder(binder).bindConfig(MemoryFileSystemCacheConfig.class); - if (isCoordinator) { - binder.bind(MemoryFileSystemCache.class).in(SINGLETON); - Provider catalogName = binder.getProvider(CatalogName.class); - newExporter(binder).export(MemoryFileSystemCache.class) - .as(generator -> generator.generatedNameOf(MemoryFileSystemCache.class, catalogName.get().toString())); - } - } -} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingBlobCache.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingBlobCache.java new file mode 100644 index 000000000000..ea11b2f66b78 --- /dev/null +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingBlobCache.java @@ -0,0 +1,75 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.filesystem.tracing; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.Tracer; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; + +import java.io.IOException; +import java.util.Collection; + +import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_LOCATION; +import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_LOCATION_COUNT; +import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_KEY; +import static io.trino.filesystem.tracing.Tracing.withTracing; +import static java.util.Objects.requireNonNull; + +public class TracingBlobCache + implements BlobCache +{ + private final Tracer tracer; + private final BlobCache delegate; + + public TracingBlobCache(Tracer tracer, BlobCache delegate) + { + this.tracer = requireNonNull(tracer, "tracer is null"); + this.delegate = requireNonNull(delegate, "delegate is null"); + } + + @Override + public Blob get(CacheKey key, BlobSource source) + throws IOException + { + Span span = tracer.spanBuilder("BlobCache.get") + .setAttribute(CACHE_FILE_LOCATION, source.toString()) + .setAttribute(CACHE_KEY, key.key()) + .startSpan(); + + return withTracing(span, () -> delegate.get(key, source)); + } + + @Override + public void invalidate(CacheKey key) + { + Span span = tracer.spanBuilder("BlobCache.invalidate") + .setAttribute(CACHE_KEY, key.key()) + .startSpan(); + + withTracing(span, () -> delegate.invalidate(key)); + } + + @Override + public void invalidate(Collection keys) + { + Span span = tracer.spanBuilder("BlobCache.invalidate") + .setAttribute(CACHE_FILE_LOCATION_COUNT, (long) keys.size()) + .startSpan(); + + withTracing(span, () -> delegate.invalidate(keys)); + } +} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystemCache.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystemCache.java deleted file mode 100644 index 9b61f5307850..000000000000 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystemCache.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.filesystem.tracing; - -import io.opentelemetry.api.trace.Span; -import io.opentelemetry.api.trace.Tracer; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoInput; -import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.TrinoInputStream; -import io.trino.filesystem.cache.TrinoFileSystemCache; - -import java.io.IOException; -import java.util.Collection; - -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_LOCATION; -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_LOCATION_COUNT; -import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_KEY; -import static io.trino.filesystem.tracing.Tracing.withTracing; -import static java.util.Objects.requireNonNull; - -public class TracingFileSystemCache - implements TrinoFileSystemCache -{ - private final Tracer tracer; - private final TrinoFileSystemCache delegate; - - public TracingFileSystemCache(Tracer tracer, TrinoFileSystemCache delegate) - { - this.tracer = requireNonNull(tracer, "tracer is null"); - this.delegate = requireNonNull(delegate, "delegate is null"); - } - - @Override - public TrinoInput cacheInput(TrinoInputFile delegate, String key) - throws IOException - { - Span span = tracer.spanBuilder("FileSystemCache.cacheInput") - .setAttribute(CACHE_FILE_LOCATION, delegate.location().toString()) - .setAttribute(CACHE_KEY, key) - .startSpan(); - - return withTracing(span, () -> this.delegate.cacheInput(delegate, key)); - } - - @Override - public TrinoInputStream cacheStream(TrinoInputFile delegate, String key) - throws IOException - { - Span span = tracer.spanBuilder("FileSystemCache.cacheStream") - .setAttribute(CACHE_FILE_LOCATION, delegate.location().toString()) - .setAttribute(CACHE_KEY, key) - .startSpan(); - - return withTracing(span, () -> this.delegate.cacheStream(delegate, key)); - } - - @Override - public long cacheLength(TrinoInputFile delegate, String key) - throws IOException - { - Span span = tracer.spanBuilder("FileSystemCache.cacheLength") - .setAttribute(CACHE_FILE_LOCATION, delegate.location().toString()) - .setAttribute(CACHE_KEY, key) - .startSpan(); - - return withTracing(span, () -> this.delegate.cacheLength(delegate, key)); - } - - @Override - public void expire(Location location) - throws IOException - { - Span span = tracer.spanBuilder("FileSystemCache.expire") - .setAttribute(CACHE_FILE_LOCATION, location.toString()) - .startSpan(); - - withTracing(span, () -> delegate.expire(location)); - } - - @Override - public void expire(Collection locations) - throws IOException - { - Span span = tracer.spanBuilder("FileSystemCache.expire") - .setAttribute(CACHE_FILE_LOCATION_COUNT, (long) locations.size()) - .startSpan(); - - withTracing(span, () -> delegate.expire(locations)); - } -} diff --git a/lib/trino-filesystem-cache-alluxio/pom.xml b/plugin/trino-blob-cache-alluxio/pom.xml similarity index 79% rename from lib/trino-filesystem-cache-alluxio/pom.xml rename to plugin/trino-blob-cache-alluxio/pom.xml index f4b8b025e8e4..cafeece9a0be 100644 --- a/lib/trino-filesystem-cache-alluxio/pom.xml +++ b/plugin/trino-blob-cache-alluxio/pom.xml @@ -9,9 +9,10 @@ ../../pom.xml - trino-filesystem-cache-alluxio + trino-blob-cache-alluxio + trino-plugin ${project.artifactId} - Trino Filesystem - Alluxio + Trino - Alluxio blob cache manager true @@ -42,37 +43,36 @@ io.airlift - configuration + bootstrap io.airlift - stats + configuration io.airlift - units + log - io.opentelemetry - opentelemetry-api + io.airlift + stats - io.trino - trino-filesystem + io.airlift + units io.trino - trino-plugin-toolkit + trino-filesystem - io.trino - trino-spi + trino-plugin-toolkit @@ -93,12 +93,6 @@ org.alluxio alluxio-core-common - - - org.slf4j - slf4j-reload4j - - @@ -106,10 +100,40 @@ jmxutils + + com.fasterxml.jackson.core + jackson-annotations + provided + + io.airlift slice - runtime + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-api-incubator + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided @@ -124,6 +148,12 @@ runtime + + org.alluxio + alluxio-core-transport + runtime + + io.airlift configuration-testing @@ -139,8 +169,7 @@ io.trino trino-filesystem - ${project.version} - tests + test-jar test diff --git a/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlob.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlob.java new file mode 100644 index 000000000000..01b7866ad58a --- /dev/null +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlob.java @@ -0,0 +1,105 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.alluxio; + +import alluxio.client.file.URIStatus; +import alluxio.client.file.cache.CacheManager; +import alluxio.conf.AlluxioConfiguration; +import io.opentelemetry.api.trace.Tracer; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobSource; + +import java.io.EOFException; +import java.io.IOException; + +import static java.util.Objects.checkFromIndexSize; +import static java.util.Objects.requireNonNull; + +final class AlluxioBlob + implements Blob +{ + private final BlobSource delegate; + private final String description; + private final long length; + private final AlluxioCacheStats statistics; + private final AlluxioInputHelper helper; + + AlluxioBlob( + Tracer tracer, + BlobSource delegate, + String cacheKey, + URIStatus status, + CacheManager cacheManager, + AlluxioConfiguration configuration, + AlluxioCacheStats statistics) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + this.description = delegate.toString(); + this.length = requireNonNull(status, "status is null").getLength(); + this.statistics = requireNonNull(statistics, "statistics is null"); + this.helper = new AlluxioInputHelper(tracer, description, cacheKey, status, cacheManager, configuration, statistics); + } + + @Override + public long length() + { + return length; + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int bufferLength) + throws IOException + { + checkFromIndexSize(offset, bufferLength, buffer.length); + if (position < 0) { + throw new IOException("Negative seek offset"); + } + if (bufferLength == 0) { + return; + } + + int bytesRead = helper.doCacheRead(position, buffer, offset, bufferLength); + if (bufferLength > bytesRead && position + bytesRead == length) { + throw new EOFException("Read %s of %s requested bytes at %s".formatted(bytesRead, bufferLength, description)); + } + doExternalRead(position + bytesRead, buffer, offset + bytesRead, bufferLength - bytesRead); + } + + private void doExternalRead(long position, byte[] buffer, int offset, int bufferLength) + throws IOException + { + if (bufferLength == 0) { + return; + } + + AlluxioInputHelper.PageAlignedRead aligned = helper.alignRead(position, bufferLength); + byte[] readBuffer = new byte[aligned.length()]; + delegate.readFully(aligned.pageStart(), readBuffer, 0, readBuffer.length); + + helper.putCache(aligned.pageStart(), aligned.pageEnd(), readBuffer, aligned.length()); + System.arraycopy(readBuffer, aligned.pageOffset(), buffer, offset, bufferLength); + statistics.recordExternalRead(readBuffer.length); + } + + @Override + public String toString() + { + return description; + } + + @Override + public void close() + { + } +} diff --git a/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCacheManager.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCacheManager.java new file mode 100644 index 000000000000..4df4a28d29d0 --- /dev/null +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCacheManager.java @@ -0,0 +1,143 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.alluxio; + +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.opentelemetry.api.trace.Tracer; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobCacheManager; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; +import io.trino.spi.catalog.CatalogName; +import org.weakref.jmx.MBeanExporter; + +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +public class AlluxioBlobCacheManager + implements BlobCacheManager +{ + private static final Logger log = Logger.get(AlluxioBlobCacheManager.class); + + private final Tracer tracer; + private final AlluxioCache sharedCache; + private final MBeanExporter exporter; + private final Map catalogs = new ConcurrentHashMap<>(); + + @Inject + public AlluxioBlobCacheManager(Tracer tracer, AlluxioCache sharedCache, MBeanExporter exporter) + { + this.tracer = requireNonNull(tracer, "tracer is null"); + this.sharedCache = requireNonNull(sharedCache, "sharedCache is null"); + this.exporter = requireNonNull(exporter, "exporter is null"); + } + + @Override + public BlobCache createBlobCache(CatalogName catalog) + { + requireNonNull(catalog, "catalog is null"); + return catalogs.computeIfAbsent(catalog, c -> { + AlluxioCacheStats stats = new AlluxioCacheStats(); + ObjectName name = statsObjectName(catalog); + CatalogEntry entry = new CatalogEntry(stats, name, new CatalogScopedBlobCache(sharedCache, catalog, tracer, stats)); + try { + exporter.export(name.getCanonicalName(), stats); + } + catch (Exception e) { + log.warn(e, "Failed to register AlluxioCacheStats MBean for catalog %s", catalog); + } + return entry; + }).blobCache(); + } + + @Override + public void drop(CatalogName catalog) + { + CatalogEntry entry = catalogs.remove(catalog); + if (entry != null) { + try { + exporter.unexport(entry.objectName().getCanonicalName()); + } + catch (Exception e) { + log.warn(e, "Failed to unregister AlluxioCacheStats MBean for catalog %s", catalog); + } + } + } + + @Override + public void shutdown() + { + Set.copyOf(catalogs.keySet()).forEach(this::drop); + try { + sharedCache.shutdown(); + } + catch (Exception e) { + log.warn(e, "Failed to shut down shared Alluxio blob cache"); + } + } + + private static ObjectName statsObjectName(CatalogName catalog) + { + try { + return ObjectName.getInstance( + "io.trino.blob.cache.alluxio:catalog=" + catalog + ",name=" + catalog + ",type=" + AlluxioCacheStats.class.getSimpleName()); + } + catch (MalformedObjectNameException e) { + throw new IllegalArgumentException("Invalid catalog name: " + catalog, e); + } + } + + private record CatalogEntry(AlluxioCacheStats stats, ObjectName objectName, BlobCache blobCache) {} + + private record CatalogScopedBlobCache(AlluxioCache delegate, CatalogName catalog, Tracer tracer, AlluxioCacheStats stats) + implements BlobCache + { + @Override + public Blob get(CacheKey key, BlobSource source) + throws IOException + { + return delegate.get(scopedKey(key), source, tracer, stats); + } + + @Override + public void invalidate(CacheKey key) + { + delegate.invalidate(scopedKey(key)); + } + + @Override + public void invalidate(Collection keys) + { + delegate.invalidate(keys.stream() + .map(this::scopedKey) + .collect(toImmutableList())); + } + + private CacheKey scopedKey(CacheKey key) + { + return new CacheKey("catalog:" + catalog + ",key:" + key.key()); + } + } +} diff --git a/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCacheManagerFactory.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCacheManagerFactory.java new file mode 100644 index 000000000000..f89fa4e8cea5 --- /dev/null +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCacheManagerFactory.java @@ -0,0 +1,86 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.alluxio; + +import alluxio.metrics.MetricsConfig; +import alluxio.metrics.MetricsSystem; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.airlift.bootstrap.Bootstrap; +import io.opentelemetry.api.trace.Tracer; +import io.trino.plugin.base.jmx.RebindSafeMBeanServer; +import io.trino.spi.cache.BlobCacheManager; +import io.trino.spi.cache.BlobCacheManagerFactory; +import io.trino.spi.cache.CacheManagerContext; +import io.trino.spi.cache.CacheLatency; +import org.weakref.jmx.MBeanExporter; + +import javax.management.MBeanServer; + +import java.lang.management.ManagementFactory; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class AlluxioBlobCacheManagerFactory + implements BlobCacheManagerFactory +{ + private static final AtomicBoolean metricsInitialized = new AtomicBoolean(); + + @Override + public String getName() + { + return "alluxio"; + } + + @Override + public CacheLatency latency() + { + return CacheLatency.DISK; + } + + @Override + public BlobCacheManager create(Map config, CacheManagerContext context) + { + MBeanServer mbeanServer = new RebindSafeMBeanServer(ManagementFactory.getPlatformMBeanServer()); + MBeanExporter mbeanExporter = new MBeanExporter(mbeanServer); + + Module module = (Binder binder) -> { + binder.bind(Tracer.class).toInstance(context.getTracer()); + binder.bind(MBeanExporter.class).toInstance(mbeanExporter); + configBinder(binder).bindConfig(AlluxioCacheConfig.class); + binder.bind(AlluxioCache.class).in(Scopes.SINGLETON); + binder.bind(AlluxioBlobCacheManager.class).in(Scopes.SINGLETON); + }; + + Bootstrap app = new Bootstrap(module) + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config); + + Injector injector = app.initialize(); + + if (metricsInitialized.compareAndSet(false, true)) { + Properties metricProps = new Properties(); + metricProps.put("sink.jmx.class", "alluxio.metrics.sink.JmxSink"); + metricProps.put("sink.jmx.domain", "org.alluxio"); + MetricsSystem.startSinksFromConfig(new MetricsConfig(metricProps)); + } + + return injector.getInstance(AlluxioBlobCacheManager.class); + } +} diff --git a/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCachePlugin.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCachePlugin.java new file mode 100644 index 000000000000..4f94db2cda5f --- /dev/null +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioBlobCachePlugin.java @@ -0,0 +1,29 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.alluxio; + +import io.trino.spi.Plugin; +import io.trino.spi.cache.BlobCacheManagerFactory; + +import java.util.List; + +public class AlluxioBlobCachePlugin + implements Plugin +{ + @Override + public Iterable getBlobCacheManagerFactories() + { + return List.of(new AlluxioBlobCacheManagerFactory()); + } +} diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCache.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCache.java similarity index 53% rename from lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCache.java rename to plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCache.java index 45e5777b280d..cdfd234327f9 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCache.java +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCache.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import alluxio.client.file.CacheContext; import alluxio.client.file.URIStatus; @@ -24,11 +24,9 @@ import com.google.inject.Inject; import io.airlift.units.DataSize; import io.opentelemetry.api.trace.Tracer; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoInput; -import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.TrinoInputStream; -import io.trino.filesystem.cache.TrinoFileSystemCache; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; import jakarta.annotation.PreDestroy; import java.io.IOException; @@ -37,55 +35,39 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; -public class AlluxioFileSystemCache - implements TrinoFileSystemCache +public class AlluxioCache { - private final Tracer tracer; private final DataSize pageSize; private final CacheManager cacheManager; private final AlluxioConfiguration config; - private final AlluxioCacheStats statistics; private final HashFunction hashFunction = Hashing.murmur3_128(); @Inject - public AlluxioFileSystemCache(Tracer tracer, AlluxioFileSystemCacheConfig config, AlluxioCacheStats statistics) + public AlluxioCache(AlluxioCacheConfig config) throws IOException { - this.tracer = requireNonNull(tracer, "tracer is null"); this.config = AlluxioConfigurationFactory.create(requireNonNull(config, "config is null")); this.pageSize = config.getCachePageSize(); this.cacheManager = CacheManager.Factory.create(this.config); - this.statistics = requireNonNull(statistics, "statistics is null"); } - @Override - public TrinoInput cacheInput(TrinoInputFile delegate, String key) + public Blob get(CacheKey key, BlobSource source, Tracer tracer, AlluxioCacheStats statistics) throws IOException { - return new AlluxioInput(tracer, delegate, key, uriStatus(delegate, key), new TracingCacheManager(tracer, key, pageSize, cacheManager), config, statistics); + requireNonNull(key, "key is null"); + requireNonNull(source, "source is null"); + URIStatus status = uriStatus(key, source); + CacheManager tracingCacheManager = new TracingCacheManager(tracer, key.key(), pageSize, cacheManager); + return new AlluxioBlob(tracer, source, key.key(), status, tracingCacheManager, config, statistics); } - @Override - public TrinoInputStream cacheStream(TrinoInputFile delegate, String key) - throws IOException - { - return new AlluxioInputStream(tracer, delegate, key, uriStatus(delegate, key), new TracingCacheManager(tracer, key, pageSize, cacheManager), config, statistics); - } - - @Override - public long cacheLength(TrinoInputFile delegate, String key) - throws IOException - { - return delegate.length(); - } - - @Override - public void expire(Location source) - throws IOException {} + // TODO: explicit invalidation is not implemented; entries are evicted by TTL and size limits. + // Implementing this requires mapping a CacheKey back to the Alluxio PageIds stored for the blob + // (see AlluxioBlobSource.pageId) and invoking CacheManager.delete for each, which needs the + // source length that is not known at invalidation time. + public void invalidate(CacheKey key) {} - @Override - public void expire(Collection locations) - throws IOException {} + public void invalidate(Collection keys) {} @PreDestroy public void shutdown() @@ -95,13 +77,13 @@ public void shutdown() } @VisibleForTesting - protected URIStatus uriStatus(TrinoInputFile file, String key) + URIStatus uriStatus(CacheKey key, BlobSource source) throws IOException { FileInfo info = new FileInfo() - .setPath(file.location().toString()) - .setLength(file.length()); - String cacheIdentifier = hashFunction.hashString(key, UTF_8).toString(); + .setPath(source.toString()) + .setLength(source.length()); + String cacheIdentifier = hashFunction.hashString(key.key(), UTF_8).toString(); return new URIStatus(info, CacheContext.defaults().setCacheIdentifier(cacheIdentifier)); } } diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCacheConfig.java similarity index 85% rename from lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java rename to plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCacheConfig.java index a45a03ace579..495cdddbc415 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheConfig.java +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCacheConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; @@ -28,7 +28,7 @@ import java.util.List; import java.util.Optional; -public class AlluxioFileSystemCacheConfig +public class AlluxioCacheConfig { static final String CACHE_DIRECTORIES = "fs.cache.directories"; static final String CACHE_MAX_SIZES = "fs.cache.max-sizes"; @@ -48,7 +48,7 @@ public List getCacheDirectories() @Config(CACHE_DIRECTORIES) @ConfigDescription("Base directory to cache data. Use a comma-separated list to cache data in multiple directories.") - public AlluxioFileSystemCacheConfig setCacheDirectories(List cacheDirectories) + public AlluxioCacheConfig setCacheDirectories(List cacheDirectories) { this.cacheDirectories = ImmutableList.copyOf(cacheDirectories); return this; @@ -61,7 +61,7 @@ public List getMaxCacheSizes() @Config(CACHE_MAX_SIZES) @ConfigDescription("The maximum cache size for a cache directory. Use a comma-separated list of sizes to specify allowed maximum values for each directory.") - public AlluxioFileSystemCacheConfig setMaxCacheSizes(List maxCacheSizes) + public AlluxioCacheConfig setMaxCacheSizes(List maxCacheSizes) { this.maxCacheSizes = ImmutableList.copyOf(maxCacheSizes); return this; @@ -75,13 +75,13 @@ public AlluxioFileSystemCacheConfig setMaxCacheSizes(List maxCacheSize @Config("fs.cache.ttl") @ConfigDescription("Duration to keep files in the cache prior to eviction") - public AlluxioFileSystemCacheConfig setCacheTTL(Duration cacheTTL) + public AlluxioCacheConfig setCacheTTL(Duration cacheTTL) { this.cacheTTL = Optional.of(cacheTTL); return this; } - public AlluxioFileSystemCacheConfig disableTTL() + public AlluxioCacheConfig disableTTL() { this.cacheTTL = Optional.empty(); return this; @@ -94,7 +94,7 @@ public AlluxioFileSystemCacheConfig disableTTL() @Config(CACHE_MAX_PERCENTAGES) @ConfigDescription("The maximum percentage (0-100) of total disk size the cache can use. Use a comma-separated list of percentage values if supplying several cache directories.") - public AlluxioFileSystemCacheConfig setMaxCacheDiskUsagePercentages(List maxCacheDiskUsagePercentages) + public AlluxioCacheConfig setMaxCacheDiskUsagePercentages(List maxCacheDiskUsagePercentages) { this.maxCacheDiskUsagePercentages = ImmutableList.copyOf(maxCacheDiskUsagePercentages); return this; @@ -110,7 +110,7 @@ public DataSize getCachePageSize() @Config("fs.cache.page-size") @ConfigDescription("Page size for cache") - public AlluxioFileSystemCacheConfig setCachePageSize(DataSize cachePageSize) + public AlluxioCacheConfig setCachePageSize(DataSize cachePageSize) { this.cachePageSize = cachePageSize; return this; diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioCacheStats.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCacheStats.java similarity index 97% rename from lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioCacheStats.java rename to plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCacheStats.java index 8ff5a424928b..38986ae50fe6 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioCacheStats.java +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioCacheStats.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import com.google.errorprone.annotations.ThreadSafe; import io.airlift.stats.DistributionStat; diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioConfigurationFactory.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioConfigurationFactory.java similarity index 92% rename from lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioConfigurationFactory.java rename to plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioConfigurationFactory.java index 1afe3dd81a6b..1b55843d8113 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioConfigurationFactory.java +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioConfigurationFactory.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import alluxio.conf.AlluxioConfiguration; import alluxio.conf.AlluxioProperties; @@ -35,16 +35,16 @@ import static alluxio.conf.PropertyKey.USER_CLIENT_CACHE_TTL_THRESHOLD_SECONDS; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.filesystem.alluxio.AlluxioFileSystemCacheConfig.CACHE_DIRECTORIES; -import static io.trino.filesystem.alluxio.AlluxioFileSystemCacheConfig.CACHE_MAX_PERCENTAGES; -import static io.trino.filesystem.alluxio.AlluxioFileSystemCacheConfig.CACHE_MAX_SIZES; +import static io.trino.blob.cache.alluxio.AlluxioCacheConfig.CACHE_DIRECTORIES; +import static io.trino.blob.cache.alluxio.AlluxioCacheConfig.CACHE_MAX_PERCENTAGES; +import static io.trino.blob.cache.alluxio.AlluxioCacheConfig.CACHE_MAX_SIZES; import static java.lang.String.join; public class AlluxioConfigurationFactory { private AlluxioConfigurationFactory() {} - public static AlluxioConfiguration create(AlluxioFileSystemCacheConfig config) + public static AlluxioConfiguration create(AlluxioCacheConfig config) { checkArgument(config.getMaxCacheSizes().isEmpty() ^ config.getMaxCacheDiskUsagePercentages().isEmpty(), "Either %s or %s must be specified", CACHE_MAX_SIZES, CACHE_MAX_PERCENTAGES); diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInputHelper.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioInputHelper.java similarity index 97% rename from lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInputHelper.java rename to plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioInputHelper.java index 209b3c8a668a..78b3fc40f136 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioInputHelper.java +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/AlluxioInputHelper.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import alluxio.client.file.CacheContext; import alluxio.client.file.URIStatus; @@ -22,7 +22,6 @@ import com.google.common.primitives.Ints; import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Tracer; -import io.trino.filesystem.Location; import java.nio.ByteBuffer; import java.util.concurrent.atomic.AtomicLong; @@ -49,7 +48,7 @@ public class AlluxioInputHelper private final String cacheKey; private final CacheManager cacheManager; private final AlluxioCacheStats statistics; - private final Location location; + private final String location; private final int pageSize; private final long fileLength; private final int bufferSize; @@ -62,7 +61,7 @@ public class AlluxioInputHelper public AlluxioInputHelper( Tracer tracer, - Location location, + String location, String cacheKey, URIStatus status, CacheManager cacheManager, @@ -87,7 +86,7 @@ public int doCacheRead(long position, byte[] bytes, int offset, int length) { Span span = tracer.spanBuilder("Alluxio.readCached") .setAttribute(CACHE_KEY, cacheKey) - .setAttribute(CACHE_FILE_LOCATION, location.toString()) + .setAttribute(CACHE_FILE_LOCATION, location) .setAttribute(CACHE_FILE_READ_SIZE, (long) length) .setAttribute(CACHE_FILE_READ_POSITION, position) .startSpan(); @@ -201,7 +200,7 @@ public void putCache(long pageStart, long pageEnd, byte[] readBuffer, int length { Span span = tracer.spanBuilder("Alluxio.writeCache") .setAttribute(CACHE_KEY, cacheKey) - .setAttribute(CACHE_FILE_LOCATION, location.toString()) + .setAttribute(CACHE_FILE_LOCATION, location) .setAttribute(CACHE_FILE_WRITE_SIZE, (long) length) .setAttribute(CACHE_FILE_WRITE_POSITION, pageStart) .startSpan(); diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/TracingCacheManager.java b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/TracingCacheManager.java similarity index 98% rename from lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/TracingCacheManager.java rename to plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/TracingCacheManager.java index 2dfec18f5cb7..93a7ae662485 100644 --- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/TracingCacheManager.java +++ b/plugin/trino-blob-cache-alluxio/src/main/java/io/trino/blob/cache/alluxio/TracingCacheManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import alluxio.client.file.CacheContext; import alluxio.client.file.cache.CacheManager; @@ -142,7 +142,7 @@ public Optional getDataFileChannel(PageId pageId, int position, Span span = tracer.spanBuilder("AlluxioCacheManager.getDataFileChannel") .setAttribute(CACHE_KEY, cacheKey) .setAttribute(CACHE_FILE_READ_POSITION, positionInFile(pageId, position)) - .setAttribute(CACHE_FILE_READ_POSITION, (long) length) + .setAttribute(CACHE_FILE_READ_SIZE, (long) length) .startSpan(); return withTracing(span, () -> delegate.getDataFileChannel(pageId, position, length, cacheContext)); } @@ -206,7 +206,7 @@ public int get(PageId pageId, int pageOffset, int bytesToRead, byte[] buffer, in .setAttribute(CACHE_FILE_READ_POSITION, positionInFile(pageId, pageOffset)) .setAttribute(CACHE_FILE_READ_SIZE, (long) bytesToRead) .startSpan(); - return withTracing(span, () -> delegate.get(pageId, bytesToRead, buffer, offsetInBuffer)); + return withTracing(span, () -> delegate.get(pageId, pageOffset, bytesToRead, buffer, offsetInBuffer)); } @Override diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/IncompleteStreamMemoryFileSystem.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/IncompleteStreamMemoryFileSystem.java similarity index 98% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/IncompleteStreamMemoryFileSystem.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/IncompleteStreamMemoryFileSystem.java index 4d789b5be23f..a05dd1edd2cd 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/IncompleteStreamMemoryFileSystem.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/IncompleteStreamMemoryFileSystem.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheConfig.java similarity index 90% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheConfig.java index 957332086ab6..d56b262cd269 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioFileSystemCacheConfig.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -27,18 +27,18 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; -import static io.trino.filesystem.alluxio.AlluxioConfigurationFactory.totalSpace; +import static io.trino.blob.cache.alluxio.AlluxioConfigurationFactory.totalSpace; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -class TestAlluxioFileSystemCacheConfig +class TestAlluxioCacheConfig { @Test public void testInvalidConfiguration() { assertThatThrownBy(() -> AlluxioConfigurationFactory.create( - new AlluxioFileSystemCacheConfig() + new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of("/cache1", "/cache2")) .setMaxCacheDiskUsagePercentages(ImmutableList.of(0)) .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("1B"))))) @@ -46,14 +46,14 @@ public void testInvalidConfiguration() .hasMessageContaining("Either fs.cache.max-sizes or fs.cache.max-disk-usage-percentages must be specified"); assertThatThrownBy(() -> AlluxioConfigurationFactory.create( - new AlluxioFileSystemCacheConfig() + new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of("/cache1", "/cache2")) .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("1B"))))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("fs.cache.directories and fs.cache.max-sizes must have the same size"); assertThatThrownBy(() -> AlluxioConfigurationFactory.create( - new AlluxioFileSystemCacheConfig() + new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of("/cache1", "/cache2")) .setMaxCacheDiskUsagePercentages(ImmutableList.of(0)))) .isInstanceOf(IllegalArgumentException.class) @@ -63,7 +63,7 @@ public void testInvalidConfiguration() @Test void testDefaults() { - assertRecordedDefaults(recordDefaults(AlluxioFileSystemCacheConfig.class) + assertRecordedDefaults(recordDefaults(AlluxioCacheConfig.class) .setCacheDirectories(ImmutableList.of()) .setCachePageSize(DataSize.valueOf("1MB")) .setMaxCacheSizes(ImmutableList.of()) @@ -85,7 +85,7 @@ public void testExplicitPropertyMappings() .put("fs.cache.ttl", "1d") .buildOrThrow(); - AlluxioFileSystemCacheConfig expected = new AlluxioFileSystemCacheConfig() + AlluxioCacheConfig expected = new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of(cacheDirectory.toString())) .setCachePageSize(DataSize.valueOf("7MB")) .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("1GB"))) diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheFileSystem.java similarity index 88% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheFileSystem.java index 232e04dae8d1..c19732874a92 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystem.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheFileSystem.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import com.google.common.collect.ImmutableList; import io.airlift.units.DataSize; @@ -32,6 +32,7 @@ import java.util.stream.Stream; import static io.airlift.tracing.Tracing.noopTracer; +import static io.trino.blob.cache.alluxio.TestingBlobCache.testingBlobCache; import static org.assertj.core.api.Assertions.assertThat; public class TestAlluxioCacheFileSystem @@ -39,7 +40,7 @@ public class TestAlluxioCacheFileSystem { private MemoryFileSystem memoryFileSystem; private CacheFileSystem fileSystem; - private AlluxioFileSystemCache cache; + private AlluxioCache cache; private Path tempDirectory; @BeforeAll @@ -49,14 +50,14 @@ void beforeAll() tempDirectory = Files.createTempDirectory("test"); Path cacheDirectory = tempDirectory.resolve("cache"); Files.createDirectory(cacheDirectory); - AlluxioFileSystemCacheConfig configuration = new AlluxioFileSystemCacheConfig() + AlluxioCacheConfig configuration = new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of(cacheDirectory.toAbsolutePath().toString())) .setCachePageSize(DataSize.valueOf("32003B")) .disableTTL() .setMaxCacheSizes(ImmutableList.of(DataSize.valueOf("100MB"))); memoryFileSystem = new IncompleteStreamMemoryFileSystem(); - cache = new AlluxioFileSystemCache(noopTracer(), configuration, new AlluxioCacheStats()); - fileSystem = new CacheFileSystem(memoryFileSystem, cache, new DefaultCacheKeyProvider()); + cache = new AlluxioCache(configuration); + fileSystem = new CacheFileSystem(memoryFileSystem, testingBlobCache(cache, noopTracer(), new AlluxioCacheStats()), new DefaultCacheKeyProvider()); } @AfterAll diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheFileSystemAccessOperations.java similarity index 95% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheFileSystemAccessOperations.java index 1601eb91bd3c..053aaadaede7 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestAlluxioCacheFileSystemAccessOperations.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestAlluxioCacheFileSystemAccessOperations.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import alluxio.client.file.cache.PageId; import alluxio.client.file.cache.PageStore; @@ -29,9 +29,11 @@ import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.cache.CacheFileSystem; +import io.trino.filesystem.cache.TrinoInputFileBlobSource; import io.trino.filesystem.memory.MemoryFileSystemFactory; -import io.trino.filesystem.tracing.TracingFileSystemCache; +import io.trino.filesystem.tracing.TracingBlobCache; import io.trino.filesystem.tracing.TracingFileSystemFactory; +import io.trino.spi.cache.BlobCache; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.TestingTelemetry; import org.junit.jupiter.api.AfterAll; @@ -51,7 +53,8 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.filesystem.alluxio.TestingCacheKeyProvider.testingCacheKeyForLocation; +import static io.trino.blob.cache.alluxio.TestingBlobCache.testingBlobCache; +import static io.trino.blob.cache.alluxio.TestingCacheKeyProvider.testingCacheKeyForLocation; import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_LOCATION; import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_READ_POSITION; import static io.trino.filesystem.tracing.CacheSystemAttributes.CACHE_FILE_READ_SIZE; @@ -75,7 +78,7 @@ public class TestAlluxioCacheFileSystemAccessOperations private final TestingTelemetry testingTelemetry = TestingTelemetry.create("alluxio-cache"); private final TestingCacheKeyProvider cacheKeyProvider = new TestingCacheKeyProvider(); private TracingFileSystemFactory tracingFileSystemFactory; - private AlluxioFileSystemCache alluxioCache; + private AlluxioCache alluxioCache; private CacheFileSystem fileSystem; private Path tempDirectory; private PageStore pageStore; @@ -87,15 +90,16 @@ public void setUp() tempDirectory = Files.createTempDirectory("test"); Path cacheDirectory = Files.createDirectory(tempDirectory.resolve("cache")); - AlluxioFileSystemCacheConfig configuration = new AlluxioFileSystemCacheConfig() + AlluxioCacheConfig configuration = new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of(cacheDirectory.toAbsolutePath().toString())) .disableTTL() .setCachePageSize(DataSize.ofBytes(PAGE_SIZE)) .setMaxCacheSizes(ImmutableList.of(DataSize.ofBytes(CACHE_SIZE))); tracingFileSystemFactory = new TracingFileSystemFactory(testingTelemetry.getTracer(), new MemoryFileSystemFactory()); - alluxioCache = new AlluxioFileSystemCache(testingTelemetry.getTracer(), configuration, new AlluxioCacheStats()); - fileSystem = new CacheFileSystem(tracingFileSystemFactory.create(ConnectorIdentity.ofUser("hello")), new TracingFileSystemCache(testingTelemetry.getTracer(), alluxioCache), cacheKeyProvider); + alluxioCache = new AlluxioCache(configuration); + BlobCache testBlobCache = new TracingBlobCache(testingTelemetry.getTracer(), testingBlobCache(alluxioCache, testingTelemetry.getTracer(), new AlluxioCacheStats())); + fileSystem = new CacheFileSystem(tracingFileSystemFactory.create(ConnectorIdentity.ofUser("hello")), testBlobCache, cacheKeyProvider); pageStore = PageStore.create(Iterables.getOnlyElement(PageStoreOptions.create(AlluxioConfigurationFactory.create(configuration)))); } @@ -284,7 +288,7 @@ public void testCacheWithMissingPage() .build()); TrinoInputFile inputFile = fileSystem.newInputFile(location); - String fileId = alluxioCache.uriStatus(inputFile, cacheKeyProvider.getCacheKey(inputFile).get()).getCacheContext().getCacheIdentifier(); + String fileId = alluxioCache.uriStatus(cacheKeyProvider.getCacheKey(inputFile).orElseThrow(), new TrinoInputFileBlobSource(inputFile)).getCacheContext().getCacheIdentifier(); // Drop this file pageStore.delete(new PageId(fileId, 0)); @@ -321,7 +325,7 @@ public void testCacheWithCorruptedPage() .build()); TrinoInputFile inputFile = fileSystem.newInputFile(location); - String fileId = alluxioCache.uriStatus(inputFile, cacheKeyProvider.getCacheKey(inputFile).get()).getCacheContext().getCacheIdentifier(); + String fileId = alluxioCache.uriStatus(cacheKeyProvider.getCacheKey(inputFile).orElseThrow(), new TrinoInputFileBlobSource(inputFile)).getCacheContext().getCacheIdentifier(); // Drop this file pageStore.put(new PageId(fileId, 0), new byte[0]); @@ -481,7 +485,7 @@ public String toString() private static String cacheKey(Location location, int cacheVersion) { - return testingCacheKeyForLocation(location, cacheVersion); + return testingCacheKeyForLocation(location, cacheVersion).key(); } private static String getLocation(SpanData span) diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestFuzzAlluxioCacheFileSystem.java similarity index 85% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestFuzzAlluxioCacheFileSystem.java index 9bc483a86906..7cf7e7c2f85f 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestFuzzAlluxioCacheFileSystem.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestFuzzAlluxioCacheFileSystem.java @@ -11,12 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import com.google.common.collect.ImmutableList; import io.airlift.slice.Slice; import io.airlift.slice.Slices; -import io.airlift.tracing.Tracing; import io.airlift.units.DataSize; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; @@ -30,10 +29,14 @@ import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; +import java.io.UncheckedIOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Comparator; import java.util.Random; +import static io.airlift.tracing.Tracing.noopTracer; +import static io.trino.blob.cache.alluxio.TestingBlobCache.testingBlobCache; import static java.lang.Math.min; import static java.lang.Math.toIntExact; import static org.assertj.core.api.Assertions.assertThat; @@ -178,13 +181,16 @@ public TrinoFileSystem create() tempDirectory = Files.createTempDirectory("test"); Path cacheDirectory = Files.createDirectory(tempDirectory.resolve("cache")); - AlluxioFileSystemCacheConfig configuration = new AlluxioFileSystemCacheConfig() + AlluxioCacheConfig configuration = new AlluxioCacheConfig() .setCacheDirectories(ImmutableList.of(cacheDirectory.toAbsolutePath().toString())) .setCachePageSize(PAGE_SIZE) .disableTTL() .setMaxCacheSizes(ImmutableList.of(CACHE_SIZE)); - AlluxioFileSystemCache alluxioCache = new AlluxioFileSystemCache(Tracing.noopTracer(), configuration, new AlluxioCacheStats()); - return new CacheFileSystem(new IncompleteStreamMemoryFileSystem(), alluxioCache, new TestingCacheKeyProvider()); + AlluxioCache alluxioCache = new AlluxioCache(configuration); + return new CacheFileSystem( + new IncompleteStreamMemoryFileSystem(), + testingBlobCache(alluxioCache, noopTracer(), new AlluxioCacheStats()), + new TestingCacheKeyProvider()); } @Override @@ -196,7 +202,23 @@ public Location tempLocation() @Override public void close() { - tempDirectory.toFile().delete(); + if (tempDirectory == null) { + return; + } + try (var paths = Files.walk(tempDirectory)) { + paths.sorted(Comparator.reverseOrder()) + .forEach(path -> { + try { + Files.delete(path); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } } } diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestTracingCacheManager.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestTracingCacheManager.java similarity index 96% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestTracingCacheManager.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestTracingCacheManager.java index fc6a823adeac..6c8d18dd50ed 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestTracingCacheManager.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestTracingCacheManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import alluxio.client.file.cache.CacheManager; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestingBlobCache.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestingBlobCache.java new file mode 100644 index 000000000000..ce919f033a50 --- /dev/null +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestingBlobCache.java @@ -0,0 +1,53 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.alluxio; + +import io.opentelemetry.api.trace.Tracer; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; + +import java.io.IOException; +import java.util.Collection; + +final class TestingBlobCache +{ + private TestingBlobCache() {} + + static BlobCache testingBlobCache(AlluxioCache cache, Tracer tracer, AlluxioCacheStats stats) + { + return new BlobCache() + { + @Override + public Blob get(CacheKey key, BlobSource source) + throws IOException + { + return cache.get(key, source, tracer, stats); + } + + @Override + public void invalidate(CacheKey key) + { + cache.invalidate(key); + } + + @Override + public void invalidate(Collection keys) + { + cache.invalidate(keys); + } + }; + } +} diff --git a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestingCacheKeyProvider.java b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestingCacheKeyProvider.java similarity index 80% rename from lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestingCacheKeyProvider.java rename to plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestingCacheKeyProvider.java index 18c29f7cdcf1..576f4ed76971 100644 --- a/lib/trino-filesystem-cache-alluxio/src/test/java/io/trino/filesystem/alluxio/TestingCacheKeyProvider.java +++ b/plugin/trino-blob-cache-alluxio/src/test/java/io/trino/blob/cache/alluxio/TestingCacheKeyProvider.java @@ -11,11 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.alluxio; +package io.trino.blob.cache.alluxio; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.cache.CacheKeyProvider; +import io.trino.spi.cache.CacheKey; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; @@ -26,14 +27,14 @@ public class TestingCacheKeyProvider private final AtomicInteger cacheVersion = new AtomicInteger(0); @Override - public Optional getCacheKey(TrinoInputFile inputFile) + public Optional getCacheKey(TrinoInputFile inputFile) { return Optional.of(testingCacheKeyForLocation(inputFile.location(), cacheVersion.get())); } - public static String testingCacheKeyForLocation(Location location, int generation) + public static CacheKey testingCacheKeyForLocation(Location location, int generation) { - return location.toString() + "-v" + generation; + return new CacheKey(location.toString() + "-v" + generation); } public void increaseCacheVersion() diff --git a/plugin/trino-blob-cache-memory/pom.xml b/plugin/trino-blob-cache-memory/pom.xml new file mode 100644 index 000000000000..ae43b1c1be5e --- /dev/null +++ b/plugin/trino-blob-cache-memory/pom.xml @@ -0,0 +1,155 @@ + + + 4.0.0 + + + io.trino + trino-root + 481-SNAPSHOT + ../../pom.xml + + + trino-blob-cache-memory + trino-plugin + ${project.artifactId} + Trino - In-memory blob cache manager + + + + com.google.guava + guava + + + + com.google.inject + guice + classes + + + + io.airlift + bootstrap + + + + io.airlift + configuration + + + + io.airlift + units + + + + io.trino + trino-cache + + + + io.trino + trino-plugin-toolkit + + + + jakarta.validation + jakarta.validation-api + + + + org.weakref + jmxutils + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + + org.checkerframework + checker-qual + runtime + + + + io.airlift + configuration-testing + test + + + + io.airlift + junit-extensions + test + + + + io.opentelemetry + opentelemetry-sdk-trace + test + + + + io.trino + trino-filesystem + test + + + + io.trino + trino-filesystem + test-jar + test + + + + io.trino + trino-testing + test + + + + org.assertj + assertj-core + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + + + diff --git a/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/InMemoryBlobCacheManager.java b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/InMemoryBlobCacheManager.java new file mode 100644 index 000000000000..c53c995af2fe --- /dev/null +++ b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/InMemoryBlobCacheManager.java @@ -0,0 +1,90 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.memory; + +import com.google.inject.Inject; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobCacheManager; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; +import io.trino.spi.catalog.CatalogName; + +import java.io.IOException; +import java.util.Collection; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +public class InMemoryBlobCacheManager + implements BlobCacheManager +{ + private final MemoryBlobCache sharedCache; + + @Inject + public InMemoryBlobCacheManager(MemoryBlobCache sharedCache) + { + this.sharedCache = requireNonNull(sharedCache, "sharedCache is null"); + } + + @Override + public BlobCache createBlobCache(CatalogName catalog) + { + requireNonNull(catalog, "catalog is null"); + return new CatalogScopedBlobCache(sharedCache, prefix(catalog)); + } + + @Override + public void drop(CatalogName catalog) + { + sharedCache.invalidatePrefix(prefix(catalog)); + } + + private static String prefix(CatalogName catalog) + { + String name = catalog.toString(); + checkArgument(!name.contains("\0"), "catalog name contains NUL byte: %s", name); + return name + "\0"; + } + + @Override + public void shutdown() + { + sharedCache.flushCache(); + } + + private record CatalogScopedBlobCache(MemoryBlobCache delegate, String prefix) + implements BlobCache + { + @Override + public Blob get(CacheKey key, BlobSource source) + throws IOException + { + return delegate.get(new CacheKey(prefix + key.key()), source); + } + + @Override + public void invalidate(CacheKey key) + { + delegate.invalidate(new CacheKey(prefix + key.key())); + } + + @Override + public void invalidate(Collection keys) + { + delegate.invalidate(keys.stream().map(k -> new CacheKey(prefix + k.key())).collect(toImmutableList())); + } + } +} diff --git a/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlob.java b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlob.java new file mode 100644 index 000000000000..b7886ee1dd87 --- /dev/null +++ b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlob.java @@ -0,0 +1,60 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.memory; + +import io.airlift.slice.Slice; +import io.trino.spi.cache.Blob; + +import java.io.EOFException; +import java.io.IOException; + +import static java.lang.Math.toIntExact; +import static java.util.Objects.checkFromIndexSize; +import static java.util.Objects.requireNonNull; + +final class MemoryBlob + implements Blob +{ + private final Slice data; + + MemoryBlob(Slice data) + { + this.data = requireNonNull(data, "data is null"); + } + + @Override + public long length() + { + return data.length(); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) + throws IOException + { + checkFromIndexSize(offset, length, buffer.length); + if (position < 0) { + throw new IOException("Negative seek offset"); + } + if (position + length > data.length()) { + throw new EOFException("Cannot read %s bytes at %s. Blob size is %s".formatted(length, position, data.length())); + } + data.getBytes(toIntExact(position), buffer, offset, length); + } + + @Override + public void close() + { + } +} diff --git a/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCache.java b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCache.java new file mode 100644 index 000000000000..d6c20651062b --- /dev/null +++ b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCache.java @@ -0,0 +1,194 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.memory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.Weigher; +import com.google.inject.Inject; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import io.trino.cache.EvictableCacheBuilder; +import io.trino.spi.cache.Blob; +import io.trino.spi.cache.BlobCache; +import io.trino.spi.cache.BlobSource; +import io.trino.spi.cache.CacheKey; +import io.trino.spi.cache.PassThroughBlob; +import org.weakref.jmx.Managed; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static io.airlift.slice.SizeOf.sizeOf; +import static io.airlift.units.DataSize.Unit.GIGABYTE; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public final class MemoryBlobCache + implements BlobCache +{ + private final Cache> cache; + private final int maxContentLengthBytes; + private final AtomicLong largeFileSkippedCount = new AtomicLong(); + + @Inject + public MemoryBlobCache(MemoryBlobCacheConfig config) + { + this(config.getCacheTtl(), config.getMaxSize(), config.getMaxContentLength()); + } + + private MemoryBlobCache(Duration expireAfterWrite, DataSize maxSize, DataSize maxContentLength) + { + checkArgument(maxContentLength.compareTo(DataSize.of(1, GIGABYTE)) <= 0, "maxContentLength must be less than or equal to 1GB"); + this.cache = EvictableCacheBuilder.newBuilder() + .maximumWeight(maxSize.toBytes()) + .weigher((Weigher>) (key, value) -> toIntExact(estimatedSizeOf(key.key()) + sizeOf(value, Slice::getRetainedSize))) + .expireAfterWrite(expireAfterWrite.toMillis(), TimeUnit.MILLISECONDS) + .shareNothingWhenDisabled() + .recordStats() + .build(); + this.maxContentLengthBytes = toIntExact(maxContentLength.toBytes()); + } + + @Override + public Blob get(CacheKey key, BlobSource source) + throws IOException + { + requireNonNull(key, "key is null"); + requireNonNull(source, "source is null"); + Optional cachedEntry = getOrLoad(key, source); + if (cachedEntry.isEmpty()) { + return new PassThroughBlob(source); + } + return new MemoryBlob(cachedEntry.get()); + } + + @Override + public void invalidate(CacheKey key) + { + requireNonNull(key, "key is null"); + cache.invalidate(key); + } + + @Override + public void invalidate(Collection keys) + { + requireNonNull(keys, "keys is null"); + cache.invalidateAll(keys); + } + + public void invalidatePrefix(String prefix) + { + List expired = cache.asMap().keySet().stream() + .filter(k -> k.key().startsWith(prefix)) + .collect(toImmutableList()); + cache.invalidateAll(expired); + } + + @Managed + public void flushCache() + { + cache.invalidateAll(); + } + + @Managed + public long getHitCount() + { + return cache.stats().hitCount(); + } + + @Managed + public long getMissCount() + { + return cache.stats().missCount(); + } + + @Managed + public long getRequestCount() + { + return cache.stats().requestCount(); + } + + @Managed + public long getLargeFileSkippedCount() + { + return largeFileSkippedCount.get(); + } + + public long size() + { + return cache.size(); + } + + public long evictionCount() + { + return cache.stats().evictionCount(); + } + + @VisibleForTesting + boolean isCached(CacheKey key) + { + Optional cachedEntry = cache.getIfPresent(key); + return cachedEntry != null && cachedEntry.isPresent(); + } + + private Optional getOrLoad(CacheKey key, BlobSource source) + throws IOException + { + Optional cached = cache.getIfPresent(key); + if (cached != null && cached.isPresent()) { + return cached; + } + long length = source.length(); + if (length > maxContentLengthBytes) { + largeFileSkippedCount.incrementAndGet(); + return Optional.empty(); + } + try { + return cache.get(key, () -> load(source, length)); + } + catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException io) { + throw io; + } + if (cause instanceof RuntimeException re) { + throw re; + } + throw new IOException(cause); + } + } + + private Optional load(BlobSource source, long length) + { + try { + byte[] buffer = new byte[toIntExact(length)]; + source.readFully(0, buffer, 0, buffer.length); + return Optional.of(Slices.wrappedBuffer(buffer)); + } + catch (IOException e) { + return Optional.empty(); + } + } +} diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCacheConfig.java b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCacheConfig.java similarity index 88% rename from lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCacheConfig.java rename to plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCacheConfig.java index 4ca8b120c398..a26d44ba9533 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystemCacheConfig.java +++ b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCacheConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.memory; +package io.trino.blob.cache.memory; import com.google.common.annotations.VisibleForTesting; import io.airlift.configuration.Config; @@ -25,7 +25,7 @@ import static io.airlift.units.DataSize.Unit.MEGABYTE; import static java.util.concurrent.TimeUnit.HOURS; -public class MemoryFileSystemCacheConfig +public class MemoryBlobCacheConfig { @VisibleForTesting static final DataSize DEFAULT_CACHE_SIZE = HeapSizeParser.DEFAULT.parse("2%"); @@ -42,7 +42,7 @@ public Duration getCacheTtl() @Config("fs.memory-cache.ttl") @ConfigDescription("Duration to keep files in the cache prior to eviction") - public MemoryFileSystemCacheConfig setCacheTtl(Duration ttl) + public MemoryBlobCacheConfig setCacheTtl(Duration ttl) { this.ttl = ttl; return this; @@ -56,7 +56,7 @@ public DataSize getMaxSize() @Config("fs.memory-cache.max-size") @ConfigDescription("Maximum total size of the cache") - public MemoryFileSystemCacheConfig setMaxSize(DataSize maxSize) + public MemoryBlobCacheConfig setMaxSize(DataSize maxSize) { this.maxSize = maxSize; return this; @@ -72,7 +72,7 @@ public DataSize getMaxContentLength() @Config("fs.memory-cache.max-content-length") @ConfigDescription("Maximum size of file that can be cached") - public MemoryFileSystemCacheConfig setMaxContentLength(DataSize maxContentLength) + public MemoryBlobCacheConfig setMaxContentLength(DataSize maxContentLength) { this.maxContentLength = maxContentLength; return this; diff --git a/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCacheManagerFactory.java b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCacheManagerFactory.java new file mode 100644 index 000000000000..53469c6cff9c --- /dev/null +++ b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCacheManagerFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.memory; + +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.airlift.bootstrap.Bootstrap; +import io.trino.spi.cache.BlobCacheManager; +import io.trino.spi.cache.BlobCacheManagerFactory; +import io.trino.spi.cache.CacheManagerContext; +import io.trino.spi.cache.CacheLatency; + +import java.util.Map; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class MemoryBlobCacheManagerFactory + implements BlobCacheManagerFactory +{ + @Override + public String getName() + { + return "memory"; + } + + @Override + public CacheLatency latency() + { + return CacheLatency.MEMORY; + } + + @Override + public BlobCacheManager create(Map config, CacheManagerContext context) + { + Module module = (Binder binder) -> { + configBinder(binder).bindConfig(MemoryBlobCacheConfig.class); + binder.bind(MemoryBlobCache.class).in(Scopes.SINGLETON); + binder.bind(InMemoryBlobCacheManager.class).in(Scopes.SINGLETON); + }; + + Bootstrap app = new Bootstrap(module) + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config); + + Injector injector = app.initialize(); + return injector.getInstance(InMemoryBlobCacheManager.class); + } +} diff --git a/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCachePlugin.java b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCachePlugin.java new file mode 100644 index 000000000000..25eca5b797f2 --- /dev/null +++ b/plugin/trino-blob-cache-memory/src/main/java/io/trino/blob/cache/memory/MemoryBlobCachePlugin.java @@ -0,0 +1,29 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.blob.cache.memory; + +import io.trino.spi.Plugin; +import io.trino.spi.cache.BlobCacheManagerFactory; + +import java.util.List; + +public class MemoryBlobCachePlugin + implements Plugin +{ + @Override + public Iterable getBlobCacheManagerFactories() + { + return List.of(new MemoryBlobCacheManagerFactory()); + } +} diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/cache/TestCacheFileSystemAccessOperations.java b/plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestCacheFileSystemAccessOperations.java similarity index 93% rename from lib/trino-filesystem/src/test/java/io/trino/filesystem/cache/TestCacheFileSystemAccessOperations.java rename to plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestCacheFileSystemAccessOperations.java index d530ab4fcc8d..2f21e6c8203f 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/cache/TestCacheFileSystemAccessOperations.java +++ b/plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestCacheFileSystemAccessOperations.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.cache; +package io.trino.blob.cache.memory; import com.google.common.collect.HashMultiset; import com.google.common.collect.ImmutableMultiset; @@ -23,8 +23,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.memory.MemoryFileSystemCache; -import io.trino.filesystem.memory.MemoryFileSystemCacheConfig; +import io.trino.filesystem.cache.CacheFileSystem; +import io.trino.filesystem.cache.DefaultCacheKeyProvider; import io.trino.filesystem.memory.MemoryFileSystemFactory; import io.trino.filesystem.tracing.TracingFileSystemFactory; import io.trino.testing.TestingTelemetry; @@ -57,9 +57,9 @@ public class TestCacheFileSystemAccessOperations void setUp() { tracingFileSystemFactory = new TracingFileSystemFactory(telemetry.getTracer(), new MemoryFileSystemFactory()); - MemoryFileSystemCacheConfig configuration = new MemoryFileSystemCacheConfig() + MemoryBlobCacheConfig configuration = new MemoryBlobCacheConfig() .setCacheTtl(new Duration(24, HOURS)); - fileSystem = new CacheFileSystem(tracingFileSystemFactory.create(TestingConnectorSession.SESSION), new MemoryFileSystemCache(configuration), new DefaultCacheKeyProvider()); + fileSystem = new CacheFileSystem(tracingFileSystemFactory.create(TestingConnectorSession.SESSION), new MemoryBlobCache(configuration), new DefaultCacheKeyProvider()); } @AfterAll diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystemCache.java b/plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestMemoryBlobCache.java similarity index 93% rename from lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystemCache.java rename to plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestMemoryBlobCache.java index b065946d6666..1f802eae9d7a 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystemCache.java +++ b/plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestMemoryBlobCache.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.memory; +package io.trino.blob.cache.memory; import io.airlift.units.DataSize; import io.airlift.units.Duration; @@ -23,6 +23,7 @@ import io.trino.filesystem.cache.CacheFileSystem; import io.trino.filesystem.cache.CacheKeyProvider; import io.trino.filesystem.cache.DefaultCacheKeyProvider; +import io.trino.filesystem.memory.MemoryFileSystem; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -34,24 +35,24 @@ import static java.util.concurrent.TimeUnit.HOURS; import static org.assertj.core.api.Assertions.assertThat; -public class TestMemoryFileSystemCache +public class TestMemoryBlobCache extends AbstractTestTrinoFileSystem { private static final int MAX_CONTENT_LENGTH = 2 * 1024 * 1024; private MemoryFileSystem delegate; private CacheFileSystem fileSystem; - private MemoryFileSystemCache cache; + private MemoryBlobCache cache; private CacheKeyProvider cacheKeyProvider; @BeforeAll void beforeAll() { - MemoryFileSystemCacheConfig configuration = new MemoryFileSystemCacheConfig() + MemoryBlobCacheConfig configuration = new MemoryBlobCacheConfig() .setMaxContentLength(DataSize.ofBytes(MAX_CONTENT_LENGTH)) .setCacheTtl(new Duration(8, HOURS)); delegate = new MemoryFileSystem(); - cache = new MemoryFileSystemCache(configuration); + cache = new MemoryBlobCache(configuration); cacheKeyProvider = new DefaultCacheKeyProvider(); fileSystem = new CacheFileSystem(delegate, cache, cacheKeyProvider); } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystemCacheConfig.java b/plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestMemoryBlobCacheConfig.java similarity index 85% rename from lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystemCacheConfig.java rename to plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestMemoryBlobCacheConfig.java index bf0648949ce1..ca2488eb3d7e 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystemCacheConfig.java +++ b/plugin/trino-blob-cache-memory/src/test/java/io/trino/blob/cache/memory/TestMemoryBlobCacheConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.filesystem.memory; +package io.trino.blob.cache.memory; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; @@ -25,15 +25,15 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static io.trino.filesystem.memory.MemoryFileSystemCacheConfig.DEFAULT_CACHE_SIZE; +import static io.trino.blob.cache.memory.MemoryBlobCacheConfig.DEFAULT_CACHE_SIZE; import static java.util.concurrent.TimeUnit.HOURS; -public class TestMemoryFileSystemCacheConfig +public class TestMemoryBlobCacheConfig { @Test void testDefaults() { - assertRecordedDefaults(recordDefaults(MemoryFileSystemCacheConfig.class) + assertRecordedDefaults(recordDefaults(MemoryBlobCacheConfig.class) .setMaxSize(DEFAULT_CACHE_SIZE) .setCacheTtl(new Duration(1, HOURS)) .setMaxContentLength(DataSize.of(8, MEGABYTE))); @@ -49,7 +49,7 @@ public void testExplicitPropertyMappings() .put("fs.memory-cache.ttl", "8h") .buildOrThrow(); - MemoryFileSystemCacheConfig expected = new MemoryFileSystemCacheConfig() + MemoryBlobCacheConfig expected = new MemoryBlobCacheConfig() .setMaxSize(DataSize.of(10, MEGABYTE)) .setCacheTtl(new Duration(8, HOURS)) .setMaxContentLength(DataSize.of(1, MEGABYTE)); diff --git a/plugin/trino-delta-lake/pom.xml b/plugin/trino-delta-lake/pom.xml index a23c3d94eb12..f9d54001975c 100644 --- a/plugin/trino-delta-lake/pom.xml +++ b/plugin/trino-delta-lake/pom.xml @@ -333,6 +333,12 @@ test + + io.trino + trino-blob-cache-alluxio + test + + io.trino trino-client 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 cb0a4f41388b..11b45f98e178 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 @@ -92,7 +92,7 @@ public class DeltaLakeConfig private boolean projectionPushdownEnabled = true; private boolean queryPartitionFilterRequired; private boolean deletionVectorsEnabled; - private boolean deltaLogFileSystemCacheDisabled; + private boolean deltaLogBlobCacheDisabled; private int metadataParallelism = 8; private int checkpointProcessingParallelism = 4; @@ -547,16 +547,16 @@ public DeltaLakeConfig setDeletionVectorsEnabled(boolean deletionVectorsEnabled) return this; } - public boolean isDeltaLogFileSystemCacheDisabled() + public boolean isDeltaLogBlobCacheDisabled() { - return deltaLogFileSystemCacheDisabled; + return deltaLogBlobCacheDisabled; } @Config("delta.fs.cache.disable-transaction-log-caching") - @ConfigDescription("Disable filesystem caching of the _delta_log directory (effective only when fs.cache.enabled=true)") - public DeltaLakeConfig setDeltaLogFileSystemCacheDisabled(boolean deltaLogFileSystemCacheDisabled) + @ConfigDescription("Disable blob caching of the _delta_log directory (effective only when fs.cache.enabled=true)") + public DeltaLakeConfig setDeltaLogBlobCacheDisabled(boolean deltaLogBlobCacheDisabled) { - this.deltaLogFileSystemCacheDisabled = deltaLogFileSystemCacheDisabled; + this.deltaLogBlobCacheDisabled = deltaLogBlobCacheDisabled; return this; } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/cache/DeltaLakeCacheKeyProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/cache/DeltaLakeCacheKeyProvider.java index e967b93b65b8..3eb260d51227 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/cache/DeltaLakeCacheKeyProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/cache/DeltaLakeCacheKeyProvider.java @@ -17,6 +17,7 @@ import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.cache.CacheKeyProvider; import io.trino.plugin.deltalake.DeltaLakeConfig; +import io.trino.spi.cache.CacheKey; import java.util.Optional; @@ -34,17 +35,17 @@ public DeltaLakeCacheKeyProvider(DeltaLakeConfig deltaLakeConfig) { // Disabling the delta log folder caching is useful in those scenarios when Delta Tables are deleted and re-created, // and caching their _delta_log directories should be avoided. - this.deltaLogCacheDisabled = deltaLakeConfig.isDeltaLogFileSystemCacheDisabled(); + this.deltaLogCacheDisabled = deltaLakeConfig.isDeltaLogBlobCacheDisabled(); } /** * Get the cache key of a TrinoInputFile. Returns Optional.empty() if the file is not cacheable. */ @Override - public Optional getCacheKey(TrinoInputFile inputFile) + public Optional getCacheKey(TrinoInputFile inputFile) { String path = inputFile.location().path(); - // Explicitly exclude the files in the _delta_log directory when deltaLogFileSystemCacheDisabled is set to true, + // Explicitly exclude the files in the _delta_log directory when deltaLogBlobCacheDisabled is set to true, // as they can change when the Delta Table is overwritten, https://github.com/trinodb/trino/issues/21451 if (deltaLogCacheDisabled && path.contains("/" + TRANSACTION_LOG_DIRECTORY + "/")) { return Optional.empty(); @@ -59,6 +60,6 @@ public Optional getCacheKey(TrinoInputFile inputFile) || path.contains("/" + STARBURST_META_DIR + "/")) { return Optional.empty(); } - return Optional.of(path); + return Optional.of(new CacheKey(inputFile.location().toString())); } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java index d73fd4c4f13d..b1822d280b92 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java @@ -22,6 +22,7 @@ import io.airlift.units.DataSize; import io.airlift.units.Duration; import io.trino.Session; +import io.trino.blob.cache.alluxio.AlluxioBlobCachePlugin; import io.trino.execution.QueryManager; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastore; @@ -31,6 +32,7 @@ import io.trino.plugin.hive.TestingHivePlugin; import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; +import io.trino.spi.Plugin; import io.trino.spi.QueryId; import io.trino.sql.planner.OptimizerConfig.JoinDistributionType; import io.trino.testing.BaseConnectorSmokeTest; @@ -50,6 +52,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CyclicBarrier; @@ -225,7 +228,7 @@ protected QueryRunner createQueryRunner() private QueryRunner createDeltaLakeQueryRunner() throws Exception { - return DeltaLakeQueryRunner.builder(SCHEMA) + DeltaLakeQueryRunner.Builder builder = DeltaLakeQueryRunner.builder(SCHEMA) .setDeltaProperties(ImmutableMap.builder() .put("hive.metastore.uri", hiveHadoop.getHiveMetastoreEndpoint().toString()) .put("delta.metadata.cache-ttl", TEST_METADATA_CACHE_TTL_SECONDS + "s") @@ -234,8 +237,27 @@ private QueryRunner createDeltaLakeQueryRunner() .put("hive.metastore.thrift.client.read-timeout", "1m") // read timed out sometimes happens with the default timeout .putAll(deltaStorageConfiguration()) .buildOrThrow()) - .setSchemaLocation(getLocationForTable(bucketName, SCHEMA)) - .build(); + .setSchemaLocation(getLocationForTable(bucketName, SCHEMA)); + getBlobCacheProperties().ifPresent(properties -> { + builder.withPlugin(getBlobCachePlugin()); + builder.withBlobCache(getBlobCacheType(), properties); + }); + return builder.build(); + } + + protected String getBlobCacheType() + { + return "alluxio"; + } + + protected Plugin getBlobCachePlugin() + { + return new AlluxioBlobCachePlugin(); + } + + protected Optional> getBlobCacheProperties() + { + return Optional.empty(); } @AfterAll diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheFileOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheFileOperations.java index 480ed8e57972..d9353906e163 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheFileOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheFileOperations.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; import com.google.common.io.Resources; +import io.trino.blob.cache.alluxio.AlluxioBlobCachePlugin; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import org.intellij.lang.annotations.Language; @@ -53,11 +54,14 @@ protected DistributedQueryRunner createQueryRunner() .setCoordinatorProperties(ImmutableMap.of("node-scheduler.include-coordinator", "false")) .setDeltaProperties(ImmutableMap.builder() .put("fs.cache.enabled", "true") - .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) - .put("fs.cache.max-sizes", "100MB") .put("delta.enable-non-concurrent-writes", "true") .put("delta.register-table-procedure.enabled", "true") .buildOrThrow()) + .withPlugin(new AlluxioBlobCachePlugin()) + .withBlobCache("alluxio", ImmutableMap.builder() + .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) + .put("fs.cache.max-sizes", "100MB") + .buildOrThrow()) .setWorkerCount(1) .build(); } @@ -90,8 +94,7 @@ public void testCacheFileOperations() .add(new CacheOperation("InputFile.length", "00000000000000000001.json")) .add(new CacheOperation("Alluxio.readCached", "00000000000000000002.json", 0, 658)) .add(new CacheOperation("InputFile.length", "00000000000000000002.json")) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000002.json", 0, 658)) - .add(new CacheOperation("InputFile.newStream", "00000000000000000002.json")) + .add(new CacheOperation("Input.readFully", "00000000000000000002.json", 0, 658)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000002.json", 0, 658)) .add(new CacheOperation("InputFile.length", "00000000000000000003.json")) .add(new CacheOperation("InputFile.newStream", "_last_checkpoint")) @@ -133,9 +136,8 @@ public void testCacheFileOperations() .add(new CacheOperation("Alluxio.readCached", "00000000000000000004.json", 0, 658)) .add(new CacheOperation("InputFile.length", "00000000000000000004.json")) .add(new CacheOperation("Alluxio.readCached", "00000000000000000005.json", 0, 658)) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000005.json", 0, 658)) + .add(new CacheOperation("Input.readFully", "00000000000000000005.json", 0, 658)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000005.json", 0, 658)) - .add(new CacheOperation("InputFile.newStream", "00000000000000000005.json")) .add(new CacheOperation("InputFile.length", "00000000000000000005.json")) .add(new CacheOperation("InputFile.length", "00000000000000000006.json")) .add(new CacheOperation("InputFile.newStream", "_last_checkpoint")) @@ -302,17 +304,15 @@ public void testTimeTravelWithLastCheckpoint() assertFileSystemAccesses( "SELECT * FROM time_travel_with_last_checkpoint FOR VERSION AS OF 1", ImmutableMultiset.builder() - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000000.json", 0, 1015)) + .add(new CacheOperation("Input.readFully", "00000000000000000000.json", 0, 1015)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000000.json", 0, 1015)) .add(new CacheOperation("Alluxio.readCached", "00000000000000000000.json", 0, 1015)) .add(new CacheOperation("InputFile.length", "00000000000000000000.json")) - .add(new CacheOperation("InputFile.newStream", "00000000000000000000.json")) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000001.json", 0, 613)) + .add(new CacheOperation("Input.readFully", "00000000000000000001.json", 0, 613)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000001.json", 0, 613)) .add(new CacheOperation("Alluxio.readCached", "00000000000000000001.json", 0, 613)) .add(new CacheOperation("InputFile.length", "00000000000000000001.json")) .add(new CacheOperation("InputFile.exists", "00000000000000000001.json")) - .add(new CacheOperation("InputFile.newStream", "00000000000000000001.json")) .add(new CacheOperation("InputFile.newStream", "_last_checkpoint")) .addCopies(new CacheOperation("Input.readFully", "data", 0, 199), 2) .addCopies(new CacheOperation("Alluxio.writeCache", "data", 0, 199), 2) @@ -360,8 +360,7 @@ public void testTimeTravelWithLastCheckpointUsingTemporalVersion() .add(new CacheOperation("Alluxio.writeCache", "00000000000000000002.json", 0, 613)) .add(new CacheOperation("Alluxio.readCached", "00000000000000000002.json", 0, 613)) .addCopies(new CacheOperation("InputFile.length", "00000000000000000002.json"), 2) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000002.json", 0, 613)) - .add(new CacheOperation("InputFile.newStream", "00000000000000000002.json")) + .add(new CacheOperation("Input.readFully", "00000000000000000002.json", 0, 613)) .addCopies(new CacheOperation("Alluxio.readCached", "data", 0, 199), 2) .build()); assertFileSystemAccesses( @@ -523,8 +522,7 @@ public void testReadV2CheckpointJson() ImmutableMultiset.builder() .add(new CacheOperation("Alluxio.readCached", "00000000000000000001.checkpoint.73a4ddb8-2bfc-40d8-b09f-1b6a0abdfb04.json", 0, 765)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000001.checkpoint.73a4ddb8-2bfc-40d8-b09f-1b6a0abdfb04.json", 0, 765)) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000001.checkpoint.73a4ddb8-2bfc-40d8-b09f-1b6a0abdfb04.json", 0, 765)) - .add(new CacheOperation("InputFile.newStream", "00000000000000000001.checkpoint.73a4ddb8-2bfc-40d8-b09f-1b6a0abdfb04.json")) + .add(new CacheOperation("Input.readFully", "00000000000000000001.checkpoint.73a4ddb8-2bfc-40d8-b09f-1b6a0abdfb04.json", 0, 765)) .add(new CacheOperation("InputFile.length", "00000000000000000001.checkpoint.73a4ddb8-2bfc-40d8-b09f-1b6a0abdfb04.json")) .add(new CacheOperation("InputFile.length", "00000000000000000002.json")) .add(new CacheOperation("InputFile.newStream", "_last_checkpoint")) @@ -610,8 +608,7 @@ public void testCreateOrReplaceTable() assertFileSystemAccesses("CREATE OR REPLACE TABLE test_create_or_replace (id VARCHAR, age INT)", ImmutableMultiset.builder() .add(new CacheOperation("Alluxio.readCached", "00000000000000000000.json", 0, 821)) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000000.json", 0, 821)) - .add(new CacheOperation("InputFile.newStream", "00000000000000000000.json")) + .add(new CacheOperation("Input.readFully", "00000000000000000000.json", 0, 821)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000000.json", 0, 821)) .add(new CacheOperation("InputFile.length", "00000000000000000000.json")) .add(new CacheOperation("InputFile.exists", "00000000000000000001.json")) @@ -637,9 +634,8 @@ public void testCreateOrReplaceTableAsSelect() "CREATE OR REPLACE TABLE test_create_or_replace_as_select AS SELECT 1 col_name", ImmutableMultiset.builder() .add(new CacheOperation("Alluxio.readCached", "00000000000000000000.json", 0, 1063)) - .add(new CacheOperation("Alluxio.readExternalStream", "00000000000000000000.json", 0, 1063)) + .add(new CacheOperation("Input.readFully", "00000000000000000000.json", 0, 1063)) .add(new CacheOperation("Alluxio.writeCache", "00000000000000000000.json", 0, 1063)) - .add(new CacheOperation("InputFile.newStream", "00000000000000000000.json")) .add(new CacheOperation("InputFile.length", "00000000000000000000.json")) .add(new CacheOperation("InputFile.length", "00000000000000000001.json")) .add(new CacheOperation("InputFile.exists", "00000000000000000001.json")) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMinioAndHmsConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMinioAndHmsConnectorSmokeTest.java index 1dc837765873..3eb23cd06f20 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMinioAndHmsConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMinioAndHmsConnectorSmokeTest.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.deltalake; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.AfterAll; @@ -21,8 +22,10 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Comparator; -import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.Stream; /** @@ -31,20 +34,25 @@ public class TestDeltaLakeAlluxioCacheMinioAndHmsConnectorSmokeTest extends TestDeltaLakeMinioAndHmsConnectorSmokeTest { - private Path cacheDirectory; + private final List cacheDirectories = new CopyOnWriteArrayList<>(); @AfterAll final void deleteDirectory() { - try (Stream walk = Files.walk(cacheDirectory)) { - Iterator iterator = walk.sorted(Comparator.reverseOrder()).iterator(); - while (iterator.hasNext()) { - Path path = iterator.next(); - Files.delete(path); + for (Path directory : ImmutableList.copyOf(cacheDirectories)) { + try (Stream walk = Files.walk(directory)) { + walk.sorted(Comparator.reverseOrder()).forEach(path -> { + try { + Files.delete(path); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } + catch (IOException e) { + throw new UncheckedIOException(e); } - } - catch (IOException e) { - throw new UncheckedIOException(e); } super.cleanUp(); } @@ -52,17 +60,26 @@ final void deleteDirectory() @Override protected Map deltaStorageConfiguration() { + return ImmutableMap.builder() + .putAll(super.deltaStorageConfiguration()) + .put("fs.cache.enabled", "true") + .buildOrThrow(); + } + + @Override + protected Optional> getBlobCacheProperties() + { + Path cacheDirectory; try { cacheDirectory = Files.createTempDirectory("cache"); } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } - return ImmutableMap.builder() - .putAll(super.deltaStorageConfiguration()) - .put("fs.cache.enabled", "true") + cacheDirectories.add(cacheDirectory); + return Optional.of(ImmutableMap.builder() .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) .put("fs.cache.max-sizes", "100MB") - .buildOrThrow(); + .buildOrThrow()); } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMutableTransactionLog.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMutableTransactionLog.java index a69d2df8a19d..3d7e7da4c2b5 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMutableTransactionLog.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAlluxioCacheMutableTransactionLog.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; +import io.trino.blob.cache.alluxio.AlluxioBlobCachePlugin; import io.trino.filesystem.tracing.CacheFileSystemTraceUtils; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -48,12 +49,15 @@ protected DistributedQueryRunner createQueryRunner() .setCoordinatorProperties(ImmutableMap.of("node-scheduler.include-coordinator", "false")) .setDeltaProperties(ImmutableMap.builder() .put("fs.cache.enabled", "true") - .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) - .put("fs.cache.max-sizes", "100MB") .put("delta.enable-non-concurrent-writes", "true") .put("delta.register-table-procedure.enabled", "true") .put("delta.fs.cache.disable-transaction-log-caching", "true") .buildOrThrow()) + .withPlugin(new AlluxioBlobCachePlugin()) + .withBlobCache("alluxio", ImmutableMap.builder() + .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) + .put("fs.cache.max-sizes", "100MB") + .buildOrThrow()) .setWorkerCount(1) .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 4a5c55667164..010d8db9bd52 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 @@ -73,7 +73,7 @@ public void testDefaults() .setProjectionPushdownEnabled(true) .setQueryPartitionFilterRequired(false) .setDeletionVectorsEnabled(false) - .setDeltaLogFileSystemCacheDisabled(false) + .setDeltaLogBlobCacheDisabled(false) .setMetadataParallelism(8) .setCheckpointProcessingParallelism(4)); } @@ -154,7 +154,7 @@ public void testExplicitPropertyMappings() .setProjectionPushdownEnabled(false) .setQueryPartitionFilterRequired(true) .setDeletionVectorsEnabled(true) - .setDeltaLogFileSystemCacheDisabled(true) + .setDeltaLogBlobCacheDisabled(true) .setMetadataParallelism(10) .setCheckpointProcessingParallelism(8); diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 05610aaa10bd..653fd9c9160d 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -383,6 +383,12 @@ test + + io.trino + trino-blob-cache-alluxio + test + + io.trino trino-exchange-filesystem diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAlluxioCacheFileOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAlluxioCacheFileOperations.java index 0d8b3955dd7a..81023522e72d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAlluxioCacheFileOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAlluxioCacheFileOperations.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; import io.opentelemetry.sdk.trace.data.SpanData; +import io.trino.blob.cache.alluxio.AlluxioBlobCachePlugin; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import org.intellij.lang.annotations.Language; @@ -55,8 +56,6 @@ protected DistributedQueryRunner createQueryRunner() Map hiveProperties = ImmutableMap.builder() .put("fs.cache.enabled", "true") - .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) - .put("fs.cache.max-sizes", "100MB") .put("hive.metastore", "file") .put("hive.metastore.catalog.dir", metastoreDirectory.toUri().toString()) .buildOrThrow(); @@ -64,6 +63,11 @@ protected DistributedQueryRunner createQueryRunner() return HiveQueryRunner.builder() .setCoordinatorProperties(ImmutableMap.of("node-scheduler.include-coordinator", "false")) .setHiveProperties(hiveProperties) + .withPlugin(new AlluxioBlobCachePlugin()) + .withBlobCache("alluxio", ImmutableMap.builder() + .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) + .put("fs.cache.max-sizes", "100MB") + .buildOrThrow()) .setWorkerCount(1) .build(); } diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 3e988eed4a1f..bde679716a28 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -546,6 +546,17 @@ test + + io.trino + trino-blob-cache-alluxio + test + + + io.trino + trino-blob-cache-memory + test + + io.trino trino-exchange-filesystem diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/cache/IcebergCacheKeyProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/cache/IcebergCacheKeyProvider.java index 7209e0dc036a..f7f6b95a2174 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/cache/IcebergCacheKeyProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/cache/IcebergCacheKeyProvider.java @@ -15,6 +15,7 @@ import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.cache.CacheKeyProvider; +import io.trino.spi.cache.CacheKey; import java.util.Optional; @@ -22,7 +23,7 @@ public class IcebergCacheKeyProvider implements CacheKeyProvider { @Override - public Optional getCacheKey(TrinoInputFile inputFile) + public Optional getCacheKey(TrinoInputFile inputFile) { String path = inputFile.location().path(); if (path.endsWith(".trinoSchema") || path.contains("/.trinoPermissions/")) { @@ -30,6 +31,6 @@ public Optional getCacheKey(TrinoInputFile inputFile) return Optional.empty(); } // Iceberg data and metadata files are immutable - return Optional.of(path); + return Optional.of(new CacheKey(path)); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 2b1564e64a51..fb4691e52812 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -232,6 +232,8 @@ protected IcebergQueryRunner.Builder createQueryRunnerBuilder() .put("iceberg.allowed-extra-properties", "extra.property.one,extra.property.two,extra.property.three,sorted_by") // Allows testing the sorting writer flushing to the file system with smaller tables .put("iceberg.writer-sort-buffer-size", "1MB") + // Some tests rely on the caching being disabled + .put("iceberg.metadata-cache.enabled", "false") .buildOrThrow()) .setInitialTables(REQUIRED_TPCH_TABLES); } @@ -8755,8 +8757,6 @@ public void testCorruptedTableLocation() .isFalse(); } - - @Test @Timeout(10) public void testNoRetryWhenMetadataFileInvalid() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java index 173680efeb01..00648d40c155 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java @@ -17,12 +17,14 @@ import com.google.common.collect.ImmutableMap; import io.minio.messages.NotificationRecords.Event; import io.trino.Session; +import io.trino.blob.cache.alluxio.AlluxioBlobCachePlugin; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveType; import io.trino.metastore.Table; import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; +import io.trino.spi.Plugin; import io.trino.testing.QueryRunner; import io.trino.testing.minio.MinioClient; import io.trino.testing.sql.TestTable; @@ -78,7 +80,7 @@ protected QueryRunner createQueryRunner() this.hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); this.hiveMinioDataLake.start(); - return IcebergQueryRunner.builder() + IcebergQueryRunner.Builder builder = IcebergQueryRunner.builder() .setIcebergProperties( ImmutableMap.builder() .put("iceberg.file-format", format.name()) @@ -102,8 +104,12 @@ protected QueryRunner createQueryRunner() .withSchemaName(schemaName) .withClonedTpchTables(REQUIRED_TPCH_TABLES) .withSchemaProperties(Map.of("location", "'s3://" + bucketName + "/" + schemaName + "'")) - .build()) - .build(); + .build()); + getBlobCacheProperties().ifPresent(properties -> { + builder.withPlugin(getBlobCachePlugin()); + builder.withBlobCache(getBlobCacheType(), properties); + }); + return builder.build(); } public Map getAdditionalIcebergProperties() @@ -111,6 +117,21 @@ public Map getAdditionalIcebergProperties() return ImmutableMap.of(); } + protected String getBlobCacheType() + { + return "alluxio"; + } + + protected Plugin getBlobCachePlugin() + { + return new AlluxioBlobCachePlugin(); + } + + protected Optional> getBlobCacheProperties() + { + return Optional.empty(); + } + @Override protected String createSchemaSql(String schemaName) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java index 9a30adbafde1..00ec327f5873 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java @@ -21,6 +21,7 @@ import io.airlift.log.Level; import io.airlift.log.Logger; import io.airlift.log.Logging; +import io.trino.blob.cache.memory.MemoryBlobCachePlugin; import io.trino.plugin.hive.TestingHivePlugin; import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; @@ -181,6 +182,8 @@ public DistributedQueryRunner build() Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); queryRunner.installPlugin(new TestingIcebergPlugin(dataDir)); + queryRunner.installPlugin(new MemoryBlobCachePlugin()); + queryRunner.loadBlobCacheManager("memory", Map.of("fs.memory-cache.max-size", "128MB")); queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", icebergProperties.buildOrThrow()); schemaInitializer.ifPresent(initializer -> initializer.accept(queryRunner)); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAlluxioCacheFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAlluxioCacheFileOperations.java index 816792893ffa..b5f026b032ec 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAlluxioCacheFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAlluxioCacheFileOperations.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; import io.opentelemetry.sdk.trace.data.SpanData; +import io.trino.blob.cache.alluxio.AlluxioBlobCachePlugin; import io.trino.plugin.iceberg.util.FileOperationUtils.FileType; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -61,17 +62,22 @@ protected DistributedQueryRunner createQueryRunner() Map icebergProperties = ImmutableMap.builder() .put("fs.cache.enabled", "true") - .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) - .put("fs.cache.max-sizes", "100MB") .put("iceberg.metadata-cache.enabled", "false") .put("hive.metastore.catalog.dir", metastoreDirectory.toUri().toString()) .buildOrThrow(); + Map cacheProperties = ImmutableMap.builder() + .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) + .put("fs.cache.max-sizes", "100MB") + .buildOrThrow(); + DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() .setSchemaInitializer(SchemaInitializer.builder() .withSchemaName(TEST_SCHEMA) .build()) .setIcebergProperties(icebergProperties) + .withPlugin(new AlluxioBlobCachePlugin()) + .withBlobCache("alluxio", cacheProperties) .setWorkerCount(0) .build(); queryRunner.execute("CREATE SCHEMA IF NOT EXISTS " + TEST_SCHEMA); @@ -95,7 +101,7 @@ public void testCacheFileOperations() .add(new CacheOperation("Alluxio.readCached", METADATA_JSON)) .addCopies(new CacheOperation("Alluxio.readCached", SNAPSHOT), 2) .add(new CacheOperation("InputFile.length", SNAPSHOT)) - .addCopies(new CacheOperation("Alluxio.readExternalStream", MANIFEST), 2) + .addCopies(new CacheOperation("Input.readFully", MANIFEST), 2) .addCopies(new CacheOperation("Alluxio.readCached", MANIFEST), 4) .addCopies(new CacheOperation("Alluxio.writeCache", MANIFEST), 2) .build()); @@ -125,7 +131,7 @@ public void testCacheFileOperations() .addCopies(new CacheOperation("Alluxio.readCached", METADATA_JSON), 2) .addCopies(new CacheOperation("Alluxio.readCached", SNAPSHOT), 2) .add(new CacheOperation("InputFile.length", SNAPSHOT)) - .addCopies(new CacheOperation("Alluxio.readExternalStream", MANIFEST), 3) + .addCopies(new CacheOperation("Input.readFully", MANIFEST), 3) .addCopies(new CacheOperation("Alluxio.readCached", MANIFEST), 10) .addCopies(new CacheOperation("Alluxio.writeCache", MANIFEST), 3) .build()); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMemoryCacheFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMemoryCacheFileOperations.java index 9709b0830a4c..0aafa041cf78 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMemoryCacheFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMemoryCacheFileOperations.java @@ -81,23 +81,23 @@ public void testCacheFileOperations() assertFileSystemAccesses( "SELECT * FROM test_cache_file_operations", ImmutableMultiset.builder() - .addCopies(new CacheOperation("Input.readTail", DATA), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheInput", DATA), 2) - .add(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON)) - .add(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT)) - .addCopies(new CacheOperation("Input.readTail", MANIFEST), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", MANIFEST), 2) + .addCopies(new CacheOperation("Input.readFully", DATA), 2) + .addCopies(new CacheOperation("BlobCache.get", DATA), 2) + .add(new CacheOperation("BlobCache.get", METADATA_JSON)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .addCopies(new CacheOperation("Input.readFully", MANIFEST), 2) + .addCopies(new CacheOperation("BlobCache.get", MANIFEST), 2) .build()); assertFileSystemAccesses( "SELECT * FROM test_cache_file_operations", ImmutableMultiset.builder() - .addCopies(new CacheOperation("FileSystemCache.cacheInput", DATA), 2) - .add(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON)) - .add(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT)) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", MANIFEST), 2) + .addCopies(new CacheOperation("BlobCache.get", DATA), 2) + .add(new CacheOperation("BlobCache.get", METADATA_JSON)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .addCopies(new CacheOperation("BlobCache.get", MANIFEST), 2) .build()); assertUpdate("INSERT INTO test_cache_file_operations VALUES ('p3', '3-xyz')", 1); @@ -107,23 +107,23 @@ public void testCacheFileOperations() assertFileSystemAccesses( "SELECT * FROM test_cache_file_operations", ImmutableMultiset.builder() - .addCopies(new CacheOperation("Input.readTail", DATA), 3) - .addCopies(new CacheOperation("FileSystemCache.cacheInput", DATA), 5) - .add(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON)) - .add(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT)) - .addCopies(new CacheOperation("Input.readTail", MANIFEST), 3) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", MANIFEST), 5) + .addCopies(new CacheOperation("Input.readFully", DATA), 3) + .addCopies(new CacheOperation("BlobCache.get", DATA), 5) + .add(new CacheOperation("BlobCache.get", METADATA_JSON)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .addCopies(new CacheOperation("Input.readFully", MANIFEST), 3) + .addCopies(new CacheOperation("BlobCache.get", MANIFEST), 5) .build()); assertFileSystemAccesses( "SELECT * FROM test_cache_file_operations", ImmutableMultiset.builder() - .addCopies(new CacheOperation("FileSystemCache.cacheInput", DATA), 5) - .add(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON)) - .add(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT)) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", MANIFEST), 5) + .addCopies(new CacheOperation("BlobCache.get", DATA), 5) + .add(new CacheOperation("BlobCache.get", METADATA_JSON)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .addCopies(new CacheOperation("BlobCache.get", MANIFEST), 5) .build()); } @@ -134,25 +134,25 @@ public void testSelectWithFilter() assertFileSystemAccesses( "SELECT * FROM test_select_with_filter WHERE col_name = 1", ImmutableMultiset.builder() - .add(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON)) - .add(new CacheOperation("Input.readTail", METADATA_JSON)) + .add(new CacheOperation("BlobCache.get", METADATA_JSON)) + .add(new CacheOperation("Input.readFully", METADATA_JSON)) .add(new CacheOperation("InputFile.length", METADATA_JSON)) - .add(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheStream", MANIFEST)) - .add(new CacheOperation("Input.readTail", MANIFEST)) - .add(new CacheOperation("FileSystemCache.cacheInput", DATA)) - .add(new CacheOperation("Input.readTail", DATA)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", MANIFEST)) + .add(new CacheOperation("Input.readFully", MANIFEST)) + .add(new CacheOperation("BlobCache.get", DATA)) + .add(new CacheOperation("Input.readFully", DATA)) .build()); assertFileSystemAccesses( "SELECT * FROM test_select_with_filter WHERE col_name = 1", ImmutableMultiset.builder() - .add(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON)) - .add(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT)) - .add(new CacheOperation("FileSystemCache.cacheStream", MANIFEST)) - .add(new CacheOperation("FileSystemCache.cacheInput", DATA)) + .add(new CacheOperation("BlobCache.get", METADATA_JSON)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", SNAPSHOT)) + .add(new CacheOperation("BlobCache.get", MANIFEST)) + .add(new CacheOperation("BlobCache.get", DATA)) .build()); } @@ -164,24 +164,24 @@ public void testJoin() assertFileSystemAccesses("SELECT name, age FROM test_join_t1 JOIN test_join_t2 ON test_join_t2.id = test_join_t1.id", ImmutableMultiset.builder() - .addCopies(new CacheOperation("Input.readTail", METADATA_JSON), 2) + .addCopies(new CacheOperation("Input.readFully", METADATA_JSON), 2) .addCopies(new CacheOperation("InputFile.length", METADATA_JSON), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT), 2) - .addCopies(new CacheOperation("Input.readTail", MANIFEST), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", MANIFEST), 4) - .addCopies(new CacheOperation("Input.readTail", DATA), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheInput", DATA), 2) + .addCopies(new CacheOperation("BlobCache.get", METADATA_JSON), 2) + .addCopies(new CacheOperation("BlobCache.get", SNAPSHOT), 2) + .addCopies(new CacheOperation("BlobCache.get", SNAPSHOT), 2) + .addCopies(new CacheOperation("Input.readFully", MANIFEST), 2) + .addCopies(new CacheOperation("BlobCache.get", MANIFEST), 4) + .addCopies(new CacheOperation("Input.readFully", DATA), 2) + .addCopies(new CacheOperation("BlobCache.get", DATA), 2) .build()); assertFileSystemAccesses("SELECT name, age FROM test_join_t1 JOIN test_join_t2 ON test_join_t2.id = test_join_t1.id", ImmutableMultiset.builder() - .addCopies(new CacheOperation("FileSystemCache.cacheStream", METADATA_JSON), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", SNAPSHOT), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheLength", SNAPSHOT), 2) - .addCopies(new CacheOperation("FileSystemCache.cacheStream", MANIFEST), 4) - .addCopies(new CacheOperation("FileSystemCache.cacheInput", DATA), 2) + .addCopies(new CacheOperation("BlobCache.get", METADATA_JSON), 2) + .addCopies(new CacheOperation("BlobCache.get", SNAPSHOT), 2) + .addCopies(new CacheOperation("BlobCache.get", SNAPSHOT), 2) + .addCopies(new CacheOperation("BlobCache.get", MANIFEST), 4) + .addCopies(new CacheOperation("BlobCache.get", DATA), 2) .build()); } @@ -195,7 +195,7 @@ private void assertFileSystemAccesses(@Language("SQL") String query, Multiset getCacheOperations() { return getQueryRunner().getSpans().stream() - .filter(span -> span.getName().startsWith("Input.") || span.getName().startsWith("InputFile.") || span.getName().startsWith("FileSystemCache.")) + .filter(span -> span.getName().startsWith("Input.") || span.getName().startsWith("InputFile.") || span.getName().startsWith("BlobCache.")) .filter(span -> !span.getName().startsWith("InputFile.newInput")) .filter(span -> !isTrinoSchemaOrPermissions(getFileLocation(span))) .map(CacheOperation::create) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetCachingConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetCachingConnectorSmokeTest.java index 71d721b7f018..330cb82bf26c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetCachingConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetCachingConnectorSmokeTest.java @@ -23,6 +23,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Map; +import java.util.Optional; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; @@ -54,9 +55,16 @@ public Map getAdditionalIcebergProperties() { return ImmutableMap.builder() .put("fs.cache.enabled", "true") + .buildOrThrow(); + } + + @Override + protected Optional> getBlobCacheProperties() + { + return Optional.of(ImmutableMap.builder() .put("fs.cache.directories", cacheDirectory.toAbsolutePath().toString()) .put("fs.cache.max-sizes", "100MB") - .buildOrThrow(); + .buildOrThrow()); } @Override diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1648307738255131376-1-ed1a5aaf-924c-4956-ad74-a5942bc863ab.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1648307738255131376-1-ed1a5aaf-924c-4956-ad74-a5942bc863ab.avro index 5c5d180247ab..42f22f2cc75f 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1648307738255131376-1-ed1a5aaf-924c-4956-ad74-a5942bc863ab.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1648307738255131376-1-ed1a5aaf-924c-4956-ad74-a5942bc863ab.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1783632012146030949-1-0bdd8872-a5d1-4a11-8afa-a1d427282c29.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1783632012146030949-1-0bdd8872-a5d1-4a11-8afa-a1d427282c29.avro index c851f8384f30..d306557ff22c 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1783632012146030949-1-0bdd8872-a5d1-4a11-8afa-a1d427282c29.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1783632012146030949-1-0bdd8872-a5d1-4a11-8afa-a1d427282c29.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1908038061182547556-1-17582660-010e-4fe2-89d0-469798adc68a.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1908038061182547556-1-17582660-010e-4fe2-89d0-469798adc68a.avro index 0334b9bcf737..89ff48f801e0 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1908038061182547556-1-17582660-010e-4fe2-89d0-469798adc68a.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-1908038061182547556-1-17582660-010e-4fe2-89d0-469798adc68a.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2085985182010370827-1-e68ef163-0672-4599-991f-136bca8cd14e.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2085985182010370827-1-e68ef163-0672-4599-991f-136bca8cd14e.avro index 592c452c0fed..bd2619c2c2bd 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2085985182010370827-1-e68ef163-0672-4599-991f-136bca8cd14e.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2085985182010370827-1-e68ef163-0672-4599-991f-136bca8cd14e.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2326703192737831382-1-9b2367ea-07a5-4b25-a9e2-4c6c666c1321.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2326703192737831382-1-9b2367ea-07a5-4b25-a9e2-4c6c666c1321.avro index f785b58a16ee..073aacf19b1b 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2326703192737831382-1-9b2367ea-07a5-4b25-a9e2-4c6c666c1321.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2326703192737831382-1-9b2367ea-07a5-4b25-a9e2-4c6c666c1321.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2658918943052245018-1-b7882118-6324-473f-bdf3-3fd9229294ad.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2658918943052245018-1-b7882118-6324-473f-bdf3-3fd9229294ad.avro index 462a01ba15e8..2e6678c70798 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2658918943052245018-1-b7882118-6324-473f-bdf3-3fd9229294ad.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-2658918943052245018-1-b7882118-6324-473f-bdf3-3fd9229294ad.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3369490910652462497-1-5e4945f2-cf2d-447f-b748-8cd03c59d987.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3369490910652462497-1-5e4945f2-cf2d-447f-b748-8cd03c59d987.avro index 7067a83c09b0..e6684f69c1e2 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3369490910652462497-1-5e4945f2-cf2d-447f-b748-8cd03c59d987.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3369490910652462497-1-5e4945f2-cf2d-447f-b748-8cd03c59d987.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3591807473350904875-1-fd84797c-e0cd-4adf-b378-7f2917da1168.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3591807473350904875-1-fd84797c-e0cd-4adf-b378-7f2917da1168.avro index 0cd88387a127..4c3ea28f4026 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3591807473350904875-1-fd84797c-e0cd-4adf-b378-7f2917da1168.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-3591807473350904875-1-fd84797c-e0cd-4adf-b378-7f2917da1168.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-5202011232056132019-1-7181f314-2ee6-4bda-9bdd-88232eaed102.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-5202011232056132019-1-7181f314-2ee6-4bda-9bdd-88232eaed102.avro index f8c938d538cf..beaae0a35d88 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-5202011232056132019-1-7181f314-2ee6-4bda-9bdd-88232eaed102.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-5202011232056132019-1-7181f314-2ee6-4bda-9bdd-88232eaed102.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-8143898955657678113-1-d402bb82-527b-4556-a3e7-f3c32a6835a8.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-8143898955657678113-1-d402bb82-527b-4556-a3e7-f3c32a6835a8.avro index a306a3ca5e66..c87655b370b1 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-8143898955657678113-1-d402bb82-527b-4556-a3e7-f3c32a6835a8.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/materialized_view/metadata/snap-8143898955657678113-1-d402bb82-527b-4556-a3e7-f3c32a6835a8.avro differ diff --git a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/source_table/metadata/snap-3545425111120995268-1-18a67dd0-3e07-4a65-aae0-432e3ab85180.avro b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/source_table/metadata/snap-3545425111120995268-1-18a67dd0-3e07-4a65-aae0-432e3ab85180.avro index d221c157d89d..f8aea80eedcd 100644 Binary files a/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/source_table/metadata/snap-3545425111120995268-1-18a67dd0-3e07-4a65-aae0-432e3ab85180.avro and b/plugin/trino-iceberg/src/test/resources/iceberg/materialized_view_expired_snapshots/source_table/metadata/snap-3545425111120995268-1-18a67dd0-3e07-4a65-aae0-432e3ab85180.avro differ diff --git a/pom.xml b/pom.xml index d2dec619e745..63eb36aaf607 100644 --- a/pom.xml +++ b/pom.xml @@ -45,7 +45,6 @@ lib/trino-filesystem lib/trino-filesystem-alluxio lib/trino-filesystem-azure - lib/trino-filesystem-cache-alluxio lib/trino-filesystem-gcs lib/trino-filesystem-manager lib/trino-filesystem-s3 @@ -63,6 +62,8 @@ plugin/trino-base-jdbc plugin/trino-bigquery plugin/trino-blackhole + plugin/trino-blob-cache-alluxio + plugin/trino-blob-cache-memory plugin/trino-cassandra plugin/trino-clickhouse plugin/trino-datasketches @@ -970,6 +971,18 @@ ${project.version} + + io.trino + trino-blob-cache-alluxio + ${project.version} + + + + io.trino + trino-blob-cache-memory + ${project.version} + + io.trino trino-cache @@ -1064,12 +1077,6 @@ ${project.version} - - io.trino - trino-filesystem-cache-alluxio - ${project.version} - - io.trino trino-filesystem-gcs diff --git a/testing/trino-faulttolerant-tests/pom.xml b/testing/trino-faulttolerant-tests/pom.xml index 93d32367c57c..51e3c89bbfa8 100644 --- a/testing/trino-faulttolerant-tests/pom.xml +++ b/testing/trino-faulttolerant-tests/pom.xml @@ -136,6 +136,12 @@ test + + io.trino + trino-blob-cache-memory + test + + io.trino trino-client diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/StandardMultinode.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/StandardMultinode.java index 647c9f577783..33d42016f88b 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/StandardMultinode.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/StandardMultinode.java @@ -32,6 +32,7 @@ import static io.trino.tests.product.launcher.env.EnvironmentContainers.COORDINATOR; import static io.trino.tests.product.launcher.env.EnvironmentContainers.WORKER; import static io.trino.tests.product.launcher.env.common.Standard.CONTAINER_TRINO_CONFIG_PROPERTIES; +import static io.trino.tests.product.launcher.env.common.Standard.CONTAINER_TRINO_ETC; import static io.trino.tests.product.launcher.env.common.Standard.createTrinoContainer; import static java.util.Objects.requireNonNull; import static org.testcontainers.utility.MountableFile.forHostPath; @@ -81,15 +82,24 @@ public List getDependencies() @Override public void extendEnvironment(Environment.Builder builder) { - builder.configureContainer(COORDINATOR, container -> container - .withCopyFileToContainer(forHostPath(configDir.getPath("multinode-master-config.properties")), CONTAINER_TRINO_CONFIG_PROPERTIES)); + builder.configureContainer(COORDINATOR, container -> withFileSystemCaching(container) + .withCopyFileToContainer(forHostPath(configDir.getPath("multinode-master-config.properties")), CONTAINER_TRINO_CONFIG_PROPERTIES) + .withCopyFileToContainer(forHostPath(configDir.getPath("cache-manager-alluxio.properties")), CONTAINER_TRINO_ETC + "/cache-manager-alluxio.properties") + .withCopyFileToContainer(forHostPath(configDir.getPath("cache-manager-memory.properties")), CONTAINER_TRINO_ETC + "/cache-manager-memory.properties")); builder.addContainers(createTrinoWorker()); } @SuppressWarnings("resource") private DockerContainer createTrinoWorker() { - return createTrinoContainer(dockerFiles, serverPackage, jdkProvider, debug, tracing, ipv6, "ghcr.io/trinodb/testing/almalinux9-oj17:" + imagesVersion, WORKER) - .withCopyFileToContainer(forHostPath(configDir.getPath("multinode-worker-config.properties")), CONTAINER_TRINO_CONFIG_PROPERTIES); + return withFileSystemCaching(createTrinoContainer(dockerFiles, serverPackage, jdkProvider, debug, tracing, ipv6, "ghcr.io/trinodb/testing/almalinux9-oj17:" + imagesVersion, WORKER) + .withCopyFileToContainer(forHostPath(configDir.getPath("multinode-worker-config.properties")), CONTAINER_TRINO_CONFIG_PROPERTIES)); + } + + private DockerContainer withFileSystemCaching(DockerContainer container) + { + return container + .withCopyFileToContainer(forHostPath(configDir.getPath("cache-manager-alluxio.properties")), CONTAINER_TRINO_ETC + "/cache-manager-alluxio.properties") + .withCopyFileToContainer(forHostPath(configDir.getPath("cache-manager-memory.properties")), CONTAINER_TRINO_ETC + "/cache-manager-memory.properties"); } } diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/cache-manager-alluxio.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/cache-manager-alluxio.properties new file mode 100644 index 000000000000..796507b2ed8c --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/cache-manager-alluxio.properties @@ -0,0 +1,3 @@ +cache-manager.name=alluxio +fs.cache.directories=/tmp/cache/alluxio +fs.cache.max-disk-usage-percentages=90 diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/cache-manager-memory.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/cache-manager-memory.properties new file mode 100644 index 000000000000..10ad8f14b826 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/cache-manager-memory.properties @@ -0,0 +1,2 @@ +cache-manager.name=memory +fs.memory-cache.max-size=1GB diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-master-config.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-master-config.properties index 47791425624e..14f45197e38e 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-master-config.properties +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-master-config.properties @@ -21,3 +21,5 @@ catalog.management=dynamic # Reduce the amount of memory used by the finished queries and task info cache query.min-expire-age=1m task.info.max-age=1m + +cache-manager.config-files=etc/cache-manager-memory.properties,etc/cache-manager-alluxio.properties diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-worker-config.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-worker-config.properties index f899c7dbbdd9..e5f38b75496f 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-worker-config.properties +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/common/standard-multinode/multinode-worker-config.properties @@ -21,3 +21,5 @@ catalog.management=dynamic # Reduce the amount of memory used by the finished queries and task info cache query.min-expire-age=1m task.info.max-age=1m + +cache-manager.config-files=etc/cache-manager-memory.properties,etc/cache-manager-alluxio.properties diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-hive-cached/hive.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-hive-cached/hive.properties index ed9b789f1367..fa493184dbba 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-hive-cached/hive.properties +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-hive-cached/hive.properties @@ -3,7 +3,5 @@ hive.config.resources=/docker/trino-product-tests/conf/trino/etc/hive-default-fs hive.metastore.uri=thrift://hadoop-master:9083 fs.hadoop.enabled=true fs.cache.enabled=true -fs.cache.directories=/tmp/cache/hive -fs.cache.max-disk-usage-percentages=90 hive.parquet.time-zone=UTC hive.rcfile.time-zone=UTC diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-iceberg-minio-cached/iceberg.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-iceberg-minio-cached/iceberg.properties index 0f0cc6d10ab0..ba9dc0a97b6b 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-iceberg-minio-cached/iceberg.properties +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-iceberg-minio-cached/iceberg.properties @@ -1,8 +1,6 @@ connector.name=iceberg hive.metastore.uri=thrift://hadoop-master:9083 fs.cache.enabled=true -fs.cache.directories=/tmp/cache/iceberg -fs.cache.max-disk-usage-percentages=90 fs.s3.enabled=true fs.hadoop.enabled=false s3.region=us-east-1 diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-minio-data-lake-cached/delta.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-minio-data-lake-cached/delta.properties index ec5e25d22ae1..90b2c5ab0636 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-minio-data-lake-cached/delta.properties +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-minio-data-lake-cached/delta.properties @@ -9,5 +9,3 @@ s3.endpoint=http://minio:9080/ s3.path-style-access=true delta.register-table-procedure.enabled=true fs.cache.enabled=true -fs.cache.directories=/tmp/cache/delta -fs.cache.max-disk-usage-percentages=90 diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/utils/CachingTestUtils.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/utils/CachingTestUtils.java index 3e28296ac6e9..1701d1c0e39e 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/utils/CachingTestUtils.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/utils/CachingTestUtils.java @@ -27,7 +27,7 @@ public static CacheStats getCacheStats(String catalog) QueryResult queryResult = onTrino().executeQuery("SELECT " + " sum(\"cachereads.alltime.count\") as cachereads, " + " sum(\"externalreads.alltime.count\") as externalreads " + - "FROM jmx.current.\"io.trino.filesystem.alluxio:catalog=" + catalog + ",name=" + catalog + ",type=alluxiocachestats\";"); + "FROM jmx.current.\"io.trino.blob.cache.alluxio:catalog=" + catalog + ",name=" + catalog + ",type=alluxiocachestats\";"); double cacheReads = (Double) getOnlyElement(queryResult.rows()) .get(queryResult.tryFindColumnIndex("cachereads").get() - 1); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index 153686e3bb6d..69e37747d5da 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -669,6 +669,14 @@ public void loadExchangeManager(String name, Map properties) } } + @Override + public void loadBlobCacheManager(String name, Map properties) + { + for (TestingTrinoServer server : servers) { + server.loadBlobCacheManager(name, properties); + } + } + @Override public void loadSpoolingManager(String name, Map properties) { @@ -738,6 +746,9 @@ public static class Builder> private boolean withTracing; private Optional exchangeType = Optional.empty(); private Optional> exchangeProperties = Optional.empty(); + private Optional blobCacheType = Optional.empty(); + private Optional> blobCacheProperties = Optional.empty(); + private final ImmutableList.Builder plugins = ImmutableList.builder(); private int workerCount = 2; private Map extraProperties = ImmutableMap.of(); private Map coordinatorProperties = ImmutableMap.of(); @@ -929,6 +940,19 @@ private SELF withExchange(String exchangeType, Optional> pro return self(); } + public SELF withBlobCache(String type, Map properties) + { + this.blobCacheType = Optional.of(type); + this.blobCacheProperties = Optional.of(ImmutableMap.copyOf(properties)); + return self(); + } + + public SELF withPlugin(Plugin plugin) + { + plugins.add(plugin); + return self(); + } + public SELF withProtocolSpooling(String encoding) { this.encoding = Optional.of(encoding); @@ -1015,6 +1039,14 @@ public void queryCompleted(QueryCompletedEvent queryCompletedEvent) } } + for (Plugin plugin : plugins.build()) { + queryRunner.installPlugin(plugin); + } + + if (blobCacheType.isPresent()) { + queryRunner.loadBlobCacheManager(blobCacheType.orElseThrow(), blobCacheProperties.orElseThrow()); + } + additionalSetup.accept(queryRunner); } catch (Throwable e) {