From b2dd150226369690b39fd8c988f91cd1633b94d9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 14 May 2024 12:34:51 +0200 Subject: [PATCH] Fix metadata processing threads leak in Big Query --- .../io/trino/plugin/bigquery/BigQueryConnectorModule.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryConnectorModule.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryConnectorModule.java index 6c96e2188db7..5b5623a020e6 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryConnectorModule.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryConnectorModule.java @@ -41,6 +41,7 @@ import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.configuration.ConditionalModule.conditionalModule; import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.plugin.base.ClosingBinder.closingBinder; import static io.trino.plugin.bigquery.BigQueryConfig.ARROW_SERIALIZATION_ENABLED; import static java.util.concurrent.Executors.newFixedThreadPool; import static java.util.stream.Collectors.toSet; @@ -100,6 +101,8 @@ protected void setup(Binder binder) newSetBinder(proxyBinder, BigQueryOptionsConfigurer.class).addBinding().to(ProxyOptionsConfigurer.class).in(Scopes.SINGLETON); newOptionalBinder(binder, ProxyTransportFactory.class).setDefault().to(ProxyTransportFactory.DefaultProxyTransportFactory.class).in(Scopes.SINGLETON); })); + + closingBinder(binder).registerExecutor(ListeningExecutorService.class); } @Provides @@ -117,6 +120,7 @@ public static BigQueryLabelFactory labelFactory(BigQueryConfig config) } @Provides + @Singleton public ListeningExecutorService provideListeningExecutor(BigQueryConfig config) { return listeningDecorator(newFixedThreadPool(config.getMetadataParallelism(), daemonThreadsNamed("big-query-%s"))); // limit parallelism