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

Avoid blocking in BigQuery split and page sources #23741

Merged
merged 5 commits into from
Oct 17, 2024
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 @@ -416,11 +416,17 @@ public long executeUpdate(ConnectorSession session, QueryJobConfiguration job)
}

public TableResult executeQuery(ConnectorSession session, String sql)
{
return executeQuery(session, sql, null);
}

public TableResult executeQuery(ConnectorSession session, String sql, Long maxResults)
{
log.debug("Execute query: %s", sql);
QueryJobConfiguration job = QueryJobConfiguration.newBuilder(sql)
.setUseQueryCache(isQueryResultsCacheEnabled(session))
.setCreateDisposition(createDisposition(session))
.setMaxResults(maxResults)
nineinchnick marked this conversation as resolved.
Show resolved Hide resolved
.build();
return execute(session, job);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import io.opentelemetry.api.OpenTelemetry;
import io.opentelemetry.api.trace.Tracer;
import io.trino.spi.NodeManager;
import io.trino.spi.catalog.CatalogName;
import io.trino.spi.connector.Connector;
import io.trino.spi.connector.ConnectorContext;
import io.trino.spi.connector.ConnectorFactory;
Expand Down Expand Up @@ -53,6 +54,7 @@ public Connector create(String catalogName, Map<String, String> config, Connecto
binder.bind(NodeManager.class).toInstance(context.getNodeManager());
binder.bind(OpenTelemetry.class).toInstance(context.getOpenTelemetry());
binder.bind(Tracer.class).toInstance(context.getTracer());
binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName));
nineinchnick marked this conversation as resolved.
Show resolved Hide resolved
});

Injector injector = app
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.google.api.gax.rpc.HeaderProvider;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.Provides;
import com.google.inject.Scopes;
import com.google.inject.Singleton;
Expand All @@ -29,11 +30,13 @@
import io.trino.plugin.bigquery.procedure.ExecuteProcedure;
import io.trino.plugin.bigquery.ptf.Query;
import io.trino.spi.NodeManager;
import io.trino.spi.catalog.CatalogName;
import io.trino.spi.function.table.ConnectorTableFunction;
import io.trino.spi.procedure.Procedure;

import java.lang.management.ManagementFactory;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

Expand All @@ -45,6 +48,7 @@
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.newCachedThreadPool;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.stream.Collectors.toSet;

Expand Down Expand Up @@ -106,6 +110,7 @@ protected void setup(Binder binder)
}));

closingBinder(binder).registerExecutor(ListeningExecutorService.class);
closingBinder(binder).registerExecutor(Key.get(ExecutorService.class, ForBigQueryPageSource.class));
}

@Provides
Expand All @@ -129,6 +134,14 @@ public ListeningExecutorService provideListeningExecutor(BigQueryConfig config)
return listeningDecorator(newFixedThreadPool(config.getMetadataParallelism(), daemonThreadsNamed("big-query-%s"))); // limit parallelism
}

@Provides
@Singleton
@ForBigQueryPageSource
public ExecutorService provideExecutor(CatalogName catalogName)
nineinchnick marked this conversation as resolved.
Show resolved Hide resolved
{
return newCachedThreadPool(daemonThreadsNamed("bigquery-" + catalogName + "-%s"));
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We probably should have an upper bound on the number of threads here, but I wasn't sure if we can come up with a constant value or should it be configurable.

There's also another executor right above, but I didn't want to mix them up. The naming can be improved here for sure, but I also didn't have good ideas.

}

/**
* Apache Arrow requires reflective access to certain Java internals prohibited since Java 17.
* Adds an error to the {@code binder} if required --add-opens is not passed to the JVM.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.stream.Collectors;

import static com.google.common.base.Preconditions.checkArgument;
Expand All @@ -43,19 +44,22 @@ public class BigQueryPageSourceProvider
private final BigQueryTypeManager typeManager;
private final int maxReadRowsRetries;
private final boolean arrowSerializationEnabled;
private final ExecutorService executor;

@Inject
public BigQueryPageSourceProvider(
BigQueryClientFactory bigQueryClientFactory,
BigQueryReadClientFactory bigQueryReadClientFactory,
BigQueryTypeManager typeManager,
BigQueryConfig config)
BigQueryConfig config,
@ForBigQueryPageSource ExecutorService executor)
{
this.bigQueryClientFactory = requireNonNull(bigQueryClientFactory, "bigQueryClientFactory is null");
this.bigQueryReadClientFactory = requireNonNull(bigQueryReadClientFactory, "bigQueryReadClientFactory is null");
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.maxReadRowsRetries = config.getMaxReadRowsRetries();
this.arrowSerializationEnabled = config.isArrowSerializationEnabled();
this.executor = requireNonNull(executor, "executor is null");
}

@Override
Expand Down Expand Up @@ -106,12 +110,14 @@ private ConnectorPageSource createStoragePageSource(ConnectorSession session, Bi
return new BigQueryStorageArrowPageSource(
typeManager,
bigQueryReadClientFactory.create(session),
executor,
maxReadRowsRetries,
split,
columnHandles);
}
return new BigQueryStorageAvroPageSource(
bigQueryReadClientFactory.create(session),
executor,
typeManager,
maxReadRowsRetries,
split,
Expand All @@ -124,6 +130,7 @@ private ConnectorPageSource createQueryPageSource(ConnectorSession session, BigQ
session,
typeManager,
bigQueryClientFactory.create(session),
executor,
table,
columnHandles,
filter);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,13 @@
import java.time.format.DateTimeFormatterBuilder;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicLong;

import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.airlift.concurrent.MoreFutures.getFutureValue;
import static io.airlift.slice.Slices.utf8Slice;
import static io.trino.plugin.bigquery.BigQueryClient.selectSql;
import static io.trino.plugin.bigquery.BigQueryTypeManager.toTrinoTimestamp;
Expand All @@ -67,6 +71,7 @@
public class BigQueryQueryPageSource
implements ConnectorPageSource
{
private static final long MAX_PAGE_ROW_COUNT = 8192L;
private static final DateTimeFormatter TIME_FORMATTER = new DateTimeFormatterBuilder()
.appendPattern("HH:mm:ss")
.optionalStart()
Expand All @@ -78,31 +83,42 @@ public class BigQueryQueryPageSource
private final List<BigQueryColumnHandle> columnHandles;
private final PageBuilder pageBuilder;
private final boolean isQueryFunction;
private final TableResult tableResult;

private final AtomicLong readTimeNanos = new AtomicLong();

private CompletableFuture<TableResult> tableResultFuture;
private TableResult tableResult;
private boolean finished;

public BigQueryQueryPageSource(
ConnectorSession session,
BigQueryTypeManager typeManager,
BigQueryClient client,
ExecutorService executor,
BigQueryTableHandle table,
List<BigQueryColumnHandle> columnHandles,
Optional<String> filter)
{
requireNonNull(session, "session is null");
this.typeManager = requireNonNull(typeManager, "typeManager is null");
requireNonNull(client, "client is null");
requireNonNull(executor, "executor is null");
requireNonNull(table, "table is null");
requireNonNull(filter, "filter is null");
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.columnHandles = requireNonNull(columnHandles, "columnHandles is null");
requireNonNull(filter, "filter is null");
this.pageBuilder = new PageBuilder(columnHandles.stream().map(BigQueryColumnHandle::trinoType).collect(toImmutableList()));
this.isQueryFunction = table.relationHandle() instanceof BigQueryQueryRelationHandle;
String sql = buildSql(
table,
client.getProjectId(),
ImmutableList.copyOf(columnHandles),
filter);
this.tableResult = client.executeQuery(session, sql);
this.tableResultFuture = CompletableFuture.supplyAsync(() -> {
long start = System.nanoTime();
TableResult result = client.executeQuery(session, sql, MAX_PAGE_ROW_COUNT);
readTimeNanos.addAndGet(System.nanoTime() - start);
return result;
}, executor);
}

private String buildSql(BigQueryTableHandle table, String projectId, List<BigQueryColumnHandle> columns, Optional<String> filter)
Expand All @@ -127,7 +143,7 @@ public long getCompletedBytes()
@Override
public long getReadTimeNanos()
{
return 0;
return readTimeNanos.get();
}

@Override
Expand All @@ -146,7 +162,25 @@ public long getMemoryUsage()
public Page getNextPage()
{
verify(pageBuilder.isEmpty());
for (FieldValueList record : tableResult.iterateAll()) {
if (tableResult == null) {
tableResult = getFutureValue(tableResultFuture);
}
else if (tableResult.hasNextPage()) {
long start = System.nanoTime();
tableResult = tableResult.getNextPage();
readTimeNanos.addAndGet(System.nanoTime() - start);
}
else {
finished = true;
return null;
}

long start = System.nanoTime();
List<FieldValueList> values = ImmutableList.copyOf(tableResult.getValues());
finished = !tableResult.hasNextPage();
readTimeNanos.addAndGet(System.nanoTime() - start);

for (FieldValueList record : values) {
pageBuilder.declarePosition();
for (int column = 0; column < columnHandles.size(); column++) {
BigQueryColumnHandle columnHandle = columnHandles.get(column);
Expand All @@ -155,7 +189,6 @@ public Page getNextPage()
appendTo(columnHandle.trinoType(), fieldValue, output);
}
}
finished = true;

nineinchnick marked this conversation as resolved.
Show resolved Hide resolved
Page page = pageBuilder.build();
pageBuilder.reset();
Expand Down Expand Up @@ -255,5 +288,15 @@ else if (type instanceof VarbinaryType) {
}

@Override
public void close() {}
public void close()
{
tableResultFuture.cancel(true);
tableResult = null;
}

@Override
public CompletableFuture<?> isBlocked()
{
return tableResultFuture;
}
}
Loading