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

Fix metadata processing threads leak in Big Query #21968

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 @@ -30,7 +30,6 @@
import io.trino.spi.NodeManager;
import io.trino.spi.function.table.ConnectorTableFunction;

import java.lang.annotation.Target;
import java.lang.management.ManagementFactory;
import java.util.Set;
import java.util.regex.Matcher;
Expand All @@ -42,11 +41,8 @@
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.lang.annotation.ElementType.CONSTRUCTOR;
import static java.lang.annotation.ElementType.FIELD;
import static java.lang.annotation.ElementType.METHOD;
import static java.lang.annotation.ElementType.PARAMETER;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.stream.Collectors.toSet;

Expand Down Expand Up @@ -105,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
Expand All @@ -122,7 +120,7 @@ public static BigQueryLabelFactory labelFactory(BigQueryConfig config)
}

@Provides
@ForBigQuery
@Singleton
public ListeningExecutorService provideListeningExecutor(BigQueryConfig config)
{
return listeningDecorator(newFixedThreadPool(config.getMetadataParallelism(), daemonThreadsNamed("big-query-%s"))); // limit parallelism
Expand Down Expand Up @@ -186,9 +184,4 @@ protected void setup(Binder binder)
}
}
}

@Target({PARAMETER, FIELD, METHOD, CONSTRUCTOR})
public @interface ForBigQuery
{
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.inject.Inject;

import static io.trino.plugin.bigquery.BigQueryConnectorModule.ForBigQuery;
import static java.util.Objects.requireNonNull;

public class DefaultBigQueryMetadataFactory
Expand All @@ -31,7 +30,7 @@ public class DefaultBigQueryMetadataFactory
public DefaultBigQueryMetadataFactory(
BigQueryClientFactory bigQueryClient,
BigQueryTypeManager typeManager,
@ForBigQuery ListeningExecutorService executorService,
ListeningExecutorService executorService,
BigQueryConfig config)
{
this.bigQueryClient = requireNonNull(bigQueryClient, "bigQueryClient is null");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import static org.assertj.core.api.Assertions.assertThat;

/**
* Use TestBigQueryTypeMapping
* @deprecated Use {@link BaseBigQueryTypeMapping}.
*/
@Deprecated
public class TestBigQueryType
Expand Down