Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Load catalogs concurrently #15295

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import io.trino.connector.system.GlobalSystemConnector;
import io.trino.metadata.Catalog;
import io.trino.metadata.CatalogManager;
import io.trino.server.ForStartup;
import io.trino.spi.TrinoException;

import javax.annotation.PreDestroy;
Expand All @@ -31,8 +32,10 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executor;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;

Expand All @@ -42,6 +45,7 @@
import static io.trino.connector.CatalogHandle.createRootCatalogHandle;
import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS;
import static io.trino.spi.StandardErrorCode.CATALOG_NOT_AVAILABLE;
import static io.trino.util.Executors.executeUntilFailure;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

Expand All @@ -55,6 +59,7 @@ private enum State { CREATED, INITIALIZED, STOPPED }

private final CatalogStore catalogStore;
private final CatalogFactory catalogFactory;
private final Executor executor;

private final Lock catalogsUpdateLock = new ReentrantLock();
private final ConcurrentMap<String, CatalogConnector> catalogs = new ConcurrentHashMap<>();
Expand All @@ -63,10 +68,11 @@ private enum State { CREATED, INITIALIZED, STOPPED }
private State state = State.CREATED;

@Inject
public CoordinatorDynamicCatalogManager(CatalogStore catalogStore, CatalogFactory catalogFactory)
public CoordinatorDynamicCatalogManager(CatalogStore catalogStore, CatalogFactory catalogFactory, @ForStartup Executor executor)
{
this.catalogStore = requireNonNull(catalogStore, "catalogStore is null");
this.catalogFactory = requireNonNull(catalogFactory, "catalogFactory is null");
this.executor = requireNonNull(executor, "executor is null");
}

@PreDestroy
Expand Down Expand Up @@ -104,12 +110,17 @@ public void loadInitialCatalogs()
checkState(state != State.STOPPED, "ConnectorManager is stopped");
state = State.INITIALIZED;

for (CatalogProperties catalog : catalogStore.getCatalogs()) {
log.info("-- Loading catalog %s --", catalog.getCatalogHandle().getCatalogName());
CatalogConnector newCatalog = catalogFactory.createCatalog(catalog);
catalogs.put(catalog.getCatalogHandle().getCatalogName(), newCatalog);
log.info("-- Added catalog %s using connector %s --", catalog.getCatalogHandle().getCatalogName(), catalog.getConnectorName());
}
executeUntilFailure(
findepi marked this conversation as resolved.
Show resolved Hide resolved
executor,
catalogStore.getCatalogs().stream()
.map(catalog -> (Callable<?>) () -> {
log.info("-- Loading catalog %s --", catalog.getCatalogHandle().getCatalogName());
CatalogConnector newCatalog = catalogFactory.createCatalog(catalog);
catalogs.put(catalog.getCatalogHandle().getCatalogName(), newCatalog);
log.info("-- Added catalog %s using connector %s --", catalog.getCatalogHandle().getCatalogName(), catalog.getConnectorName());
return null;
})
.collect(toImmutableList()));
}
finally {
catalogsUpdateLock.unlock();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.trino.connector.system.GlobalSystemConnector;
import io.trino.metadata.Catalog;
import io.trino.metadata.CatalogManager;
import io.trino.server.ForStartup;
import io.trino.spi.TrinoException;

import javax.annotation.PreDestroy;
Expand All @@ -37,8 +38,10 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicReference;

import static com.google.common.base.MoreObjects.firstNonNull;
Expand All @@ -49,6 +52,7 @@
import static io.trino.connector.CatalogHandle.createRootCatalogHandle;
import static io.trino.spi.StandardErrorCode.CATALOG_NOT_AVAILABLE;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.util.Executors.executeUntilFailure;
import static java.util.Objects.requireNonNull;

@ThreadSafe
Expand All @@ -61,13 +65,14 @@ private enum State { CREATED, INITIALIZED, STOPPED }

private final CatalogFactory catalogFactory;
private final List<CatalogProperties> catalogProperties;
private final Executor executor;

private final ConcurrentMap<String, CatalogConnector> catalogs = new ConcurrentHashMap<>();

private final AtomicReference<State> state = new AtomicReference<>(State.CREATED);

@Inject
public StaticCatalogManager(CatalogFactory catalogFactory, StaticCatalogManagerConfig config)
public StaticCatalogManager(CatalogFactory catalogFactory, StaticCatalogManagerConfig config, @ForStartup Executor executor)
{
this.catalogFactory = requireNonNull(catalogFactory, "catalogFactory is null");
List<String> disabledCatalogs = firstNonNull(config.getDisabledCatalogs(), ImmutableList.of());
Expand Down Expand Up @@ -95,6 +100,7 @@ public StaticCatalogManager(CatalogFactory catalogFactory, StaticCatalogManagerC
catalogProperties.add(new CatalogProperties(createRootCatalogHandle(catalogName), connectorName, ImmutableMap.copyOf(properties)));
}
this.catalogProperties = catalogProperties.build();
this.executor = requireNonNull(executor, "executor is null");
}

private static List<File> listCatalogFiles(File catalogsDirectory)
Expand Down Expand Up @@ -133,13 +139,18 @@ public void loadInitialCatalogs()
return;
}

for (CatalogProperties catalog : catalogProperties) {
String catalogName = catalog.getCatalogHandle().getCatalogName();
log.info("-- Loading catalog %s --", catalogName);
CatalogConnector newCatalog = catalogFactory.createCatalog(catalog);
catalogs.put(catalogName, newCatalog);
log.info("-- Added catalog %s using connector %s --", catalogName, catalog.getConnectorName());
}
executeUntilFailure(
executor,
catalogProperties.stream()
.map(catalog -> (Callable<?>) () -> {
String catalogName = catalog.getCatalogHandle().getCatalogName();
log.info("-- Loading catalog %s --", catalogName);
CatalogConnector newCatalog = catalogFactory.createCatalog(catalog);
catalogs.put(catalogName, newCatalog);
log.info("-- Added catalog %s using connector %s --", catalogName, catalog.getConnectorName());
return null;
})
.collect(toImmutableList()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,7 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Verify.verify;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.MoreFutures.getFutureValue;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.connector.CatalogServiceProviderModule.createAccessControlProvider;
Expand Down Expand Up @@ -357,7 +358,7 @@ private LocalQueryRunner(
this.optimizerConfig = new OptimizerConfig();
LazyCatalogFactory catalogFactory = new LazyCatalogFactory();
this.catalogFactory = catalogFactory;
this.catalogManager = new CoordinatorDynamicCatalogManager(NO_STORED_CATALOGS, catalogFactory);
this.catalogManager = new CoordinatorDynamicCatalogManager(NO_STORED_CATALOGS, catalogFactory, directExecutor());
this.transactionManager = InMemoryTransactionManager.create(
new TransactionManagerConfig().setIdleTimeout(new Duration(1, TimeUnit.DAYS)),
yieldExecutor,
Expand Down