diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index 3c3bafebf56e..622e2bc998a2 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -1,42 +1,20 @@
-
+
## Description
-
-
-
-
-> Is this change a fix, improvement, new feature, refactoring, or other?
-
-> Is this a change to the core query engine, a connector, client library, or the SPI interfaces? (be specific)
-
-> How would you describe this change to a non-technical end user or system administrator?
-
-## Related issues, pull requests, and links
-
-
+
+## Non-technical explanation
-## Documentation
-( ) No documentation is needed.
-( ) Sufficient documentation is included in this PR.
-( ) Documentation PR is available with #prnumber.
-( ) Documentation issue #issuenumber is filed, and can be handled later.
+
## Release notes
-( ) No release notes entries required.
-( ) Release notes entries required with the following suggested text:
+( ) This is not user-visible and no release notes are required.
+( ) Release notes are required, please propose a release note for me.
+( ) Release notes are required, with the following suggested text:
```markdown
# Section
diff --git a/client/trino-cli/pom.xml b/client/trino-cli/pom.xml
index 72a6696e7dcc..67529ddc8522 100644
--- a/client/trino-cli/pom.xml
+++ b/client/trino-cli/pom.xml
@@ -5,7 +5,7 @@
io.trino
trino-root
- 395-SNAPSHOT
+ 396-SNAPSHOT
../../pom.xml
diff --git a/client/trino-cli/src/main/java/io/trino/cli/ClientOptions.java b/client/trino-cli/src/main/java/io/trino/cli/ClientOptions.java
index 182ca96830d1..8bdf65a1f94d 100644
--- a/client/trino-cli/src/main/java/io/trino/cli/ClientOptions.java
+++ b/client/trino-cli/src/main/java/io/trino/cli/ClientOptions.java
@@ -39,7 +39,6 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Strings.nullToEmpty;
import static io.trino.client.KerberosUtil.defaultCredentialCachePath;
-import static java.util.Collections.emptyMap;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static picocli.CommandLine.Option;
@@ -219,27 +218,25 @@ public String getKeyMap()
public ClientSession toClientSession()
{
- return new ClientSession(
- parseServer(server),
- user,
- sessionUser,
- source,
- Optional.ofNullable(traceToken),
- parseClientTags(nullToEmpty(clientTags)),
- clientInfo,
- catalog,
- schema,
- null,
- timeZone,
- Locale.getDefault(),
- toResourceEstimates(resourceEstimates),
- toProperties(sessionProperties),
- emptyMap(),
- emptyMap(),
- toExtraCredentials(extraCredentials),
- null,
- clientRequestTimeout,
- disableCompression);
+ return ClientSession.builder()
+ .server(parseServer(server))
+ .principal(user)
+ .user(sessionUser)
+ .source(source)
+ .traceToken(Optional.ofNullable(traceToken))
+ .clientTags(parseClientTags(nullToEmpty(clientTags)))
+ .clientInfo(clientInfo)
+ .catalog(catalog)
+ .schema(schema)
+ .timeZone(timeZone)
+ .locale(Locale.getDefault())
+ .resourceEstimates(toResourceEstimates(resourceEstimates))
+ .properties(toProperties(sessionProperties))
+ .credentials(toExtraCredentials(extraCredentials))
+ .transactionId(null)
+ .clientRequestTimeout(clientRequestTimeout)
+ .compressionDisabled(disableCompression)
+ .build();
}
public static URI parseServer(String server)
diff --git a/client/trino-cli/src/main/java/io/trino/cli/Console.java b/client/trino-cli/src/main/java/io/trino/cli/Console.java
index 180d7b7f0f35..b79a347c881f 100644
--- a/client/trino-cli/src/main/java/io/trino/cli/Console.java
+++ b/client/trino-cli/src/main/java/io/trino/cli/Console.java
@@ -366,8 +366,8 @@ private static boolean process(
// update catalog and schema if present
if (query.getSetCatalog().isPresent() || query.getSetSchema().isPresent()) {
session = ClientSession.builder(session)
- .withCatalog(query.getSetCatalog().orElse(session.getCatalog()))
- .withSchema(query.getSetSchema().orElse(session.getSchema()))
+ .catalog(query.getSetCatalog().orElse(session.getCatalog()))
+ .schema(query.getSetSchema().orElse(session.getSchema()))
.build();
}
@@ -379,12 +379,12 @@ private static boolean process(
ClientSession.Builder builder = ClientSession.builder(session);
if (query.getStartedTransactionId() != null) {
- builder = builder.withTransactionId(query.getStartedTransactionId());
+ builder = builder.transactionId(query.getStartedTransactionId());
}
// update path if present
if (query.getSetPath().isPresent()) {
- builder = builder.withPath(query.getSetPath().get());
+ builder = builder.path(query.getSetPath().get());
}
// update session properties if present
@@ -392,14 +392,14 @@ private static boolean process(
Map sessionProperties = new HashMap<>(session.getProperties());
sessionProperties.putAll(query.getSetSessionProperties());
sessionProperties.keySet().removeAll(query.getResetSessionProperties());
- builder = builder.withProperties(sessionProperties);
+ builder = builder.properties(sessionProperties);
}
// update session roles
if (!query.getSetRoles().isEmpty()) {
Map roles = new HashMap<>(session.getRoles());
roles.putAll(query.getSetRoles());
- builder = builder.withRoles(roles);
+ builder = builder.roles(roles);
}
// update prepared statements if present
@@ -407,7 +407,7 @@ private static boolean process(
Map preparedStatements = new HashMap<>(session.getPreparedStatements());
preparedStatements.putAll(query.getAddedPreparedStatements());
preparedStatements.keySet().removeAll(query.getDeallocatedPreparedStatements());
- builder = builder.withPreparedStatements(preparedStatements);
+ builder = builder.preparedStatements(preparedStatements);
}
session = builder.build();
diff --git a/client/trino-cli/src/main/java/io/trino/cli/OutputHandler.java b/client/trino-cli/src/main/java/io/trino/cli/OutputHandler.java
index 8a13cf6ed2e0..73a2ecd3bed5 100644
--- a/client/trino-cli/src/main/java/io/trino/cli/OutputHandler.java
+++ b/client/trino-cli/src/main/java/io/trino/cli/OutputHandler.java
@@ -94,7 +94,7 @@ public void processRows(StatementClient client)
if (row == END_TOKEN) {
break;
}
- else if (row != null) {
+ if (row != null) {
rowBuffer.add(row);
}
}
diff --git a/client/trino-cli/src/test/java/io/trino/cli/TestQueryRunner.java b/client/trino-cli/src/test/java/io/trino/cli/TestQueryRunner.java
index 536e23700595..e7acc3e0fc22 100644
--- a/client/trino-cli/src/test/java/io/trino/cli/TestQueryRunner.java
+++ b/client/trino-cli/src/test/java/io/trino/cli/TestQueryRunner.java
@@ -14,8 +14,6 @@
package io.trino.cli;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
import io.airlift.json.JsonCodec;
import io.airlift.units.Duration;
import io.trino.client.ClientSession;
@@ -102,27 +100,19 @@ public void testCookie()
static ClientSession createClientSession(MockWebServer server)
{
- return new ClientSession(
- server.url("/").uri(),
- Optional.of("user"),
- Optional.empty(),
- "source",
- Optional.empty(),
- ImmutableSet.of(),
- "clientInfo",
- "catalog",
- "schema",
- null,
- ZoneId.of("America/Los_Angeles"),
- Locale.ENGLISH,
- ImmutableMap.of(),
- ImmutableMap.of(),
- ImmutableMap.of(),
- ImmutableMap.of(),
- ImmutableMap.of(),
- null,
- new Duration(2, MINUTES),
- true);
+ return ClientSession.builder()
+ .server(server.url("/").uri())
+ .principal(Optional.of("user"))
+ .source("source")
+ .clientInfo("clientInfo")
+ .catalog("catalog")
+ .schema("schema")
+ .timeZone(ZoneId.of("America/Los_Angeles"))
+ .locale(Locale.ENGLISH)
+ .transactionId(null)
+ .clientRequestTimeout(new Duration(2, MINUTES))
+ .compressionDisabled(true)
+ .build();
}
static String createResults(MockWebServer server)
diff --git a/client/trino-client/pom.xml b/client/trino-client/pom.xml
index 55c35a139283..f0f43a825022 100644
--- a/client/trino-client/pom.xml
+++ b/client/trino-client/pom.xml
@@ -5,7 +5,7 @@
io.trino
trino-root
- 395-SNAPSHOT
+ 396-SNAPSHOT
../../pom.xml
diff --git a/client/trino-client/src/main/java/io/trino/client/ClientSession.java b/client/trino-client/src/main/java/io/trino/client/ClientSession.java
index 06819da0ef35..8fd96c6c7e53 100644
--- a/client/trino-client/src/main/java/io/trino/client/ClientSession.java
+++ b/client/trino-client/src/main/java/io/trino/client/ClientSession.java
@@ -54,6 +54,11 @@ public class ClientSession
private final Duration clientRequestTimeout;
private final boolean compressionDisabled;
+ public static Builder builder()
+ {
+ return new Builder();
+ }
+
public static Builder builder(ClientSession clientSession)
{
return new Builder(clientSession);
@@ -62,11 +67,11 @@ public static Builder builder(ClientSession clientSession)
public static ClientSession stripTransactionId(ClientSession session)
{
return ClientSession.builder(session)
- .withoutTransactionId()
+ .transactionId(null)
.build();
}
- public ClientSession(
+ private ClientSession(
URI server,
Optional principal,
Optional user,
@@ -89,8 +94,8 @@ public ClientSession(
boolean compressionDisabled)
{
this.server = requireNonNull(server, "server is null");
- this.principal = principal;
- this.user = user;
+ this.principal = requireNonNull(principal, "principal is null");
+ this.user = requireNonNull(user, "user is null");
this.source = source;
this.traceToken = requireNonNull(traceToken, "traceToken is null");
this.clientTags = ImmutableSet.copyOf(requireNonNull(clientTags, "clientTags is null"));
@@ -270,26 +275,28 @@ public String toString()
public static final class Builder
{
private URI server;
- private Optional principal;
- private Optional user;
+ private Optional principal = Optional.empty();
+ private Optional user = Optional.empty();
private String source;
- private Optional traceToken;
- private Set clientTags;
+ private Optional traceToken = Optional.empty();
+ private Set clientTags = ImmutableSet.of();
private String clientInfo;
private String catalog;
private String schema;
private String path;
private ZoneId timeZone;
private Locale locale;
- private Map resourceEstimates;
- private Map properties;
- private Map preparedStatements;
- private Map roles;
- private Map credentials;
+ private Map resourceEstimates = ImmutableMap.of();
+ private Map properties = ImmutableMap.of();
+ private Map preparedStatements = ImmutableMap.of();
+ private Map roles = ImmutableMap.of();
+ private Map credentials = ImmutableMap.of();
private String transactionId;
private Duration clientRequestTimeout;
private boolean compressionDisabled;
+ private Builder() {}
+
private Builder(ClientSession clientSession)
{
requireNonNull(clientSession, "clientSession is null");
@@ -315,61 +322,121 @@ private Builder(ClientSession clientSession)
compressionDisabled = clientSession.isCompressionDisabled();
}
- public Builder withCatalog(String catalog)
+ public Builder server(URI server)
+ {
+ this.server = server;
+ return this;
+ }
+
+ public Builder user(Optional user)
+ {
+ this.user = user;
+ return this;
+ }
+
+ public Builder principal(Optional principal)
+ {
+ this.principal = principal;
+ return this;
+ }
+
+ public Builder source(String source)
+ {
+ this.source = source;
+ return this;
+ }
+
+ public Builder traceToken(Optional traceToken)
+ {
+ this.traceToken = traceToken;
+ return this;
+ }
+
+ public Builder clientTags(Set clientTags)
+ {
+ this.clientTags = clientTags;
+ return this;
+ }
+
+ public Builder clientInfo(String clientInfo)
+ {
+ this.clientInfo = clientInfo;
+ return this;
+ }
+
+ public Builder catalog(String catalog)
+ {
+ this.catalog = catalog;
+ return this;
+ }
+
+ public Builder schema(String schema)
+ {
+ this.schema = schema;
+ return this;
+ }
+
+ public Builder path(String path)
+ {
+ this.path = path;
+ return this;
+ }
+
+ public Builder timeZone(ZoneId timeZone)
{
- this.catalog = requireNonNull(catalog, "catalog is null");
+ this.timeZone = timeZone;
return this;
}
- public Builder withSchema(String schema)
+ public Builder locale(Locale locale)
{
- this.schema = requireNonNull(schema, "schema is null");
+ this.locale = locale;
return this;
}
- public Builder withPath(String path)
+ public Builder resourceEstimates(Map resourceEstimates)
{
- this.path = requireNonNull(path, "path is null");
+ this.resourceEstimates = resourceEstimates;
return this;
}
- public Builder withProperties(Map properties)
+ public Builder properties(Map properties)
{
- this.properties = requireNonNull(properties, "properties is null");
+ this.properties = properties;
return this;
}
- public Builder withRoles(Map roles)
+ public Builder roles(Map roles)
{
this.roles = roles;
return this;
}
- public Builder withCredentials(Map credentials)
+ public Builder credentials(Map credentials)
{
- this.credentials = requireNonNull(credentials, "credentials is null");
+ this.credentials = credentials;
return this;
}
- public Builder withPreparedStatements(Map preparedStatements)
+ public Builder preparedStatements(Map preparedStatements)
{
- this.preparedStatements = requireNonNull(preparedStatements, "preparedStatements is null");
+ this.preparedStatements = preparedStatements;
return this;
}
- public Builder withTransactionId(String transactionId)
+ public Builder transactionId(String transactionId)
{
- this.transactionId = requireNonNull(transactionId, "transactionId is null");
+ this.transactionId = transactionId;
return this;
}
- public Builder withoutTransactionId()
+ public Builder clientRequestTimeout(Duration clientRequestTimeout)
{
- this.transactionId = null;
+ this.clientRequestTimeout = clientRequestTimeout;
return this;
}
- public Builder withCompressionDisabled(boolean compressionDisabled)
+ public Builder compressionDisabled(boolean compressionDisabled)
{
this.compressionDisabled = compressionDisabled;
return this;
diff --git a/client/trino-jdbc/pom.xml b/client/trino-jdbc/pom.xml
index 1bbcbab656af..ef88af985efa 100644
--- a/client/trino-jdbc/pom.xml
+++ b/client/trino-jdbc/pom.xml
@@ -5,7 +5,7 @@
io.trino
trino-root
- 395-SNAPSHOT
+ 396-SNAPSHOT
../../pom.xml
diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java
index c841627b37f4..b2675de1180b 100644
--- a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java
+++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java
@@ -733,27 +733,27 @@ StatementClient startQuery(String sql, Map sessionPropertiesOver
int millis = networkTimeoutMillis.get();
Duration timeout = (millis > 0) ? new Duration(millis, MILLISECONDS) : new Duration(999, DAYS);
- ClientSession session = new ClientSession(
- httpUri,
- user,
- sessionUser,
- source,
- Optional.ofNullable(clientInfo.get(TRACE_TOKEN)),
- ImmutableSet.copyOf(clientTags),
- clientInfo.get(CLIENT_INFO),
- catalog.get(),
- schema.get(),
- path.get(),
- timeZoneId.get(),
- locale.get(),
- ImmutableMap.of(),
- ImmutableMap.copyOf(allProperties),
- ImmutableMap.copyOf(preparedStatements),
- ImmutableMap.copyOf(roles),
- extraCredentials,
- transactionId.get(),
- timeout,
- compressionDisabled);
+ ClientSession session = ClientSession.builder()
+ .server(httpUri)
+ .principal(user)
+ .user(sessionUser)
+ .source(source)
+ .traceToken(Optional.ofNullable(clientInfo.get(TRACE_TOKEN)))
+ .clientTags(ImmutableSet.copyOf(clientTags))
+ .clientInfo(clientInfo.get(CLIENT_INFO))
+ .catalog(catalog.get())
+ .schema(schema.get())
+ .path(path.get())
+ .timeZone(timeZoneId.get())
+ .locale(locale.get())
+ .properties(ImmutableMap.copyOf(allProperties))
+ .preparedStatements(ImmutableMap.copyOf(preparedStatements))
+ .roles(ImmutableMap.copyOf(roles))
+ .credentials(extraCredentials)
+ .transactionId(transactionId.get())
+ .clientRequestTimeout(timeout)
+ .compressionDisabled(compressionDisabled)
+ .build();
return newStatementClient(httpClient, session, sql);
}
diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml
index 24b6512d88d9..c59081575ce5 100644
--- a/core/trino-main/pom.xml
+++ b/core/trino-main/pom.xml
@@ -5,7 +5,7 @@
io.trino
trino-root
- 395-SNAPSHOT
+ 396-SNAPSHOT
../../pom.xml
diff --git a/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java b/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java
index dfaeea21c274..58bc37d5c921 100644
--- a/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java
+++ b/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java
@@ -280,12 +280,10 @@ private static LocalCostEstimate calculateJoinExchangeCost(
LocalCostEstimate localRepartitionCost = calculateLocalRepartitionCost(buildSizeInBytes);
return addPartialComponents(replicateCost, localRepartitionCost);
}
- else {
- LocalCostEstimate probeCost = calculateRemoteRepartitionCost(probeSizeInBytes);
- LocalCostEstimate buildRemoteRepartitionCost = calculateRemoteRepartitionCost(buildSizeInBytes);
- LocalCostEstimate buildLocalRepartitionCost = calculateLocalRepartitionCost(buildSizeInBytes);
- return addPartialComponents(probeCost, buildRemoteRepartitionCost, buildLocalRepartitionCost);
- }
+ LocalCostEstimate probeCost = calculateRemoteRepartitionCost(probeSizeInBytes);
+ LocalCostEstimate buildRemoteRepartitionCost = calculateRemoteRepartitionCost(buildSizeInBytes);
+ LocalCostEstimate buildLocalRepartitionCost = calculateLocalRepartitionCost(buildSizeInBytes);
+ return addPartialComponents(probeCost, buildRemoteRepartitionCost, buildLocalRepartitionCost);
}
public static LocalCostEstimate calculateJoinInputCost(
diff --git a/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java b/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java
index ec6ab4c6b196..9075b27d6e23 100644
--- a/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java
+++ b/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java
@@ -312,20 +312,18 @@ private SymbolStatsEstimate estimateCoalesce(SymbolStatsEstimate left, SymbolSta
if (left.getNullsFraction() == 0) {
return left;
}
- else if (left.getNullsFraction() == 1.0) {
+ if (left.getNullsFraction() == 1.0) {
return right;
}
- else {
- return SymbolStatsEstimate.builder()
- .setLowValue(min(left.getLowValue(), right.getLowValue()))
- .setHighValue(max(left.getHighValue(), right.getHighValue()))
- .setDistinctValuesCount(left.getDistinctValuesCount() +
- min(right.getDistinctValuesCount(), input.getOutputRowCount() * left.getNullsFraction()))
- .setNullsFraction(left.getNullsFraction() * right.getNullsFraction())
- // TODO check if dataSize estimation method is correct
- .setAverageRowSize(max(left.getAverageRowSize(), right.getAverageRowSize()))
- .build();
- }
+ return SymbolStatsEstimate.builder()
+ .setLowValue(min(left.getLowValue(), right.getLowValue()))
+ .setHighValue(max(left.getHighValue(), right.getHighValue()))
+ .setDistinctValuesCount(left.getDistinctValuesCount() +
+ min(right.getDistinctValuesCount(), input.getOutputRowCount() * left.getNullsFraction()))
+ .setNullsFraction(left.getNullsFraction() * right.getNullsFraction())
+ // TODO check if dataSize estimation method is correct
+ .setAverageRowSize(max(left.getAverageRowSize(), right.getAverageRowSize()))
+ .build();
}
}
diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java b/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java
index 5f31eef50fd2..79e7a0c83a6b 100644
--- a/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java
+++ b/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java
@@ -391,15 +391,13 @@ public QueryResults getQueryResults(long token, UriInfo uriInfo)
DispatchInfo.queued(NO_DURATION, NO_DURATION));
}
- Optional dispatchInfo = dispatchManager.getDispatchInfo(queryId);
- if (dispatchInfo.isEmpty()) {
- // query should always be found, but it may have just been determined to be abandoned
- throw new WebApplicationException(Response
- .status(NOT_FOUND)
- .build());
- }
+ DispatchInfo dispatchInfo = dispatchManager.getDispatchInfo(queryId)
+ // query should always be found, but it may have just been determined to be abandoned
+ .orElseThrow(() -> new WebApplicationException(Response
+ .status(NOT_FOUND)
+ .build()));
- return createQueryResults(token + 1, uriInfo, dispatchInfo.get());
+ return createQueryResults(token + 1, uriInfo, dispatchInfo);
}
public void cancel()
diff --git a/core/trino-main/src/main/java/io/trino/exchange/LazyExchangeDataSource.java b/core/trino-main/src/main/java/io/trino/exchange/LazyExchangeDataSource.java
index 6325ba91afb9..58e697d7bf71 100644
--- a/core/trino-main/src/main/java/io/trino/exchange/LazyExchangeDataSource.java
+++ b/core/trino-main/src/main/java/io/trino/exchange/LazyExchangeDataSource.java
@@ -116,6 +116,7 @@ public void addInput(ExchangeInput input)
return;
}
ExchangeDataSource dataSource = delegate.get();
+ boolean inputAdded = false;
if (dataSource == null) {
if (input instanceof DirectExchangeInput) {
DirectExchangeClient client = directExchangeClientSupplier.get(queryId, exchangeId, systemMemoryContext, taskFailureListener, retryPolicy);
@@ -126,7 +127,8 @@ else if (input instanceof SpoolingExchangeInput) {
ExchangeManager exchangeManager = exchangeManagerRegistry.getExchangeManager();
List sourceHandles = spoolingExchangeInput.getExchangeSourceHandles();
ExchangeSource exchangeSource = exchangeManager.createSource(sourceHandles);
- dataSource = new SpoolingExchangeDataSource(exchangeSource, sourceHandles, systemMemoryContext);
+ dataSource = new SpoolingExchangeDataSource(exchangeSource, systemMemoryContext);
+ inputAdded = true;
}
else {
throw new IllegalArgumentException("Unexpected input: " + input);
@@ -134,7 +136,9 @@ else if (input instanceof SpoolingExchangeInput) {
delegate.set(dataSource);
initialized = true;
}
- dataSource.addInput(input);
+ if (!inputAdded) {
+ dataSource.addInput(input);
+ }
}
if (initialized) {
diff --git a/core/trino-main/src/main/java/io/trino/exchange/SpoolingExchangeDataSource.java b/core/trino-main/src/main/java/io/trino/exchange/SpoolingExchangeDataSource.java
index 51b5cc624444..62a44ea83c4f 100644
--- a/core/trino-main/src/main/java/io/trino/exchange/SpoolingExchangeDataSource.java
+++ b/core/trino-main/src/main/java/io/trino/exchange/SpoolingExchangeDataSource.java
@@ -13,18 +13,13 @@
*/
package io.trino.exchange;
-import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.log.Logger;
import io.airlift.slice.Slice;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.operator.OperatorInfo;
import io.trino.spi.exchange.ExchangeSource;
-import io.trino.spi.exchange.ExchangeSourceHandle;
-import java.util.List;
-
-import static com.google.common.base.Preconditions.checkState;
import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.airlift.concurrent.MoreFutures.toListenableFuture;
import static java.util.Objects.requireNonNull;
@@ -39,18 +34,13 @@ public class SpoolingExchangeDataSource
// It doesn't have to be declared as volatile as the nullification of this variable doesn't have to be immediately visible to other threads.
// However since close can be called at any moment this variable has to be accessed in a safe way (avoiding "check-then-use").
private ExchangeSource exchangeSource;
- private final List exchangeSourceHandles;
private final LocalMemoryContext systemMemoryContext;
private volatile boolean closed;
- public SpoolingExchangeDataSource(
- ExchangeSource exchangeSource,
- List exchangeSourceHandles,
- LocalMemoryContext systemMemoryContext)
+ public SpoolingExchangeDataSource(ExchangeSource exchangeSource, LocalMemoryContext systemMemoryContext)
{
// this assignment is expected to be followed by an assignment of a final field to ensure safe publication
this.exchangeSource = requireNonNull(exchangeSource, "exchangeSource is null");
- this.exchangeSourceHandles = ImmutableList.copyOf(requireNonNull(exchangeSourceHandles, "exchangeSourceHandles is null"));
this.systemMemoryContext = requireNonNull(systemMemoryContext, "systemMemoryContext is null");
}
@@ -96,16 +86,7 @@ public ListenableFuture isBlocked()
@Override
public void addInput(ExchangeInput input)
{
- SpoolingExchangeInput exchangeInput = (SpoolingExchangeInput) input;
- // Only a single input is expected when the spooling exchange is used.
- // The engine adds the same input to every instance of the ExchangeOperator.
- // Since the ExchangeDataSource is shared between ExchangeOperator instances
- // the same input may be delivered multiple times.
- checkState(
- exchangeInput.getExchangeSourceHandles().equals(exchangeSourceHandles),
- "exchange input is expected to contain an identical exchangeSourceHandles list: %s != %s",
- exchangeInput.getExchangeSourceHandles(),
- exchangeSourceHandles);
+ throw new UnsupportedOperationException("only a single input is expected");
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/CommentTask.java b/core/trino-main/src/main/java/io/trino/execution/CommentTask.java
index bf408789aa9d..cd29320675dd 100644
--- a/core/trino-main/src/main/java/io/trino/execution/CommentTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/CommentTask.java
@@ -30,7 +30,6 @@
import java.util.List;
import java.util.Map;
-import java.util.Optional;
import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.trino.metadata.MetadataUtil.createQualifiedObjectName;
@@ -133,12 +132,10 @@ else if (metadata.getTableHandle(session, viewName).isPresent()) {
private void commentOnColumn(Comment statement, Session session)
{
- Optional prefix = statement.getName().getPrefix();
- if (prefix.isEmpty()) {
- throw semanticException(MISSING_TABLE, statement, "Table must be specified");
- }
+ QualifiedName prefix = statement.getName().getPrefix()
+ .orElseThrow(() -> semanticException(MISSING_TABLE, statement, "Table must be specified"));
- QualifiedObjectName originalTableName = createQualifiedObjectName(session, statement, prefix.get());
+ QualifiedObjectName originalTableName = createQualifiedObjectName(session, statement, prefix);
RedirectionAwareTableHandle redirectionAwareTableHandle = metadata.getRedirectionAwareTableHandle(session, originalTableName);
if (redirectionAwareTableHandle.getTableHandle().isEmpty()) {
throw semanticException(TABLE_NOT_FOUND, statement, "Table does not exist: " + originalTableName);
diff --git a/core/trino-main/src/main/java/io/trino/execution/CommitTask.java b/core/trino-main/src/main/java/io/trino/execution/CommitTask.java
index f777048525c4..43438e9adb7b 100644
--- a/core/trino-main/src/main/java/io/trino/execution/CommitTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/CommitTask.java
@@ -54,10 +54,7 @@ public ListenableFuture execute(
WarningCollector warningCollector)
{
Session session = stateMachine.getSession();
- if (session.getTransactionId().isEmpty()) {
- throw new TrinoException(NOT_IN_TRANSACTION, "No transaction in progress");
- }
- TransactionId transactionId = session.getTransactionId().get();
+ TransactionId transactionId = session.getTransactionId().orElseThrow(() -> new TrinoException(NOT_IN_TRANSACTION, "No transaction in progress"));
stateMachine.clearTransactionId();
return transactionManager.asyncCommit(transactionId);
diff --git a/core/trino-main/src/main/java/io/trino/execution/DataDefinitionExecution.java b/core/trino-main/src/main/java/io/trino/execution/DataDefinitionExecution.java
index 9aacb600f157..521926d8dd6b 100644
--- a/core/trino-main/src/main/java/io/trino/execution/DataDefinitionExecution.java
+++ b/core/trino-main/src/main/java/io/trino/execution/DataDefinitionExecution.java
@@ -165,7 +165,7 @@ public void onFailure(Throwable throwable)
}
@Override
- public void addOutputInfoListener(Consumer listener)
+ public void setOutputInfoListener(Consumer listener)
{
// DDL does not have an output
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/DenyTask.java b/core/trino-main/src/main/java/io/trino/execution/DenyTask.java
index 2a4cd0032a16..943c03d84210 100644
--- a/core/trino-main/src/main/java/io/trino/execution/DenyTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/DenyTask.java
@@ -19,7 +19,6 @@
import io.trino.metadata.Metadata;
import io.trino.metadata.QualifiedObjectName;
import io.trino.metadata.RedirectionAwareTableHandle;
-import io.trino.metadata.TableHandle;
import io.trino.security.AccessControl;
import io.trino.spi.connector.CatalogSchemaName;
import io.trino.spi.security.Privilege;
@@ -97,8 +96,7 @@ private static void executeDenyOnTable(Session session, Deny statement, Metadata
{
QualifiedObjectName tableName = createQualifiedObjectName(session, statement, statement.getName());
RedirectionAwareTableHandle redirection = metadata.getRedirectionAwareTableHandle(session, tableName);
- Optional tableHandle = redirection.getTableHandle();
- if (tableHandle.isEmpty()) {
+ if (redirection.getTableHandle().isEmpty()) {
throw semanticException(TABLE_NOT_FOUND, statement, "Table '%s' does not exist", tableName);
}
if (redirection.getRedirectedTableName().isPresent()) {
diff --git a/core/trino-main/src/main/java/io/trino/execution/GrantTask.java b/core/trino-main/src/main/java/io/trino/execution/GrantTask.java
index ad5d2cbf2ba7..806912b3f8d9 100644
--- a/core/trino-main/src/main/java/io/trino/execution/GrantTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/GrantTask.java
@@ -19,7 +19,6 @@
import io.trino.metadata.Metadata;
import io.trino.metadata.QualifiedObjectName;
import io.trino.metadata.RedirectionAwareTableHandle;
-import io.trino.metadata.TableHandle;
import io.trino.security.AccessControl;
import io.trino.spi.connector.CatalogSchemaName;
import io.trino.spi.security.Privilege;
@@ -101,8 +100,7 @@ private void executeGrantOnTable(Session session, Grant statement)
{
QualifiedObjectName tableName = createQualifiedObjectName(session, statement, statement.getName());
RedirectionAwareTableHandle redirection = metadata.getRedirectionAwareTableHandle(session, tableName);
- Optional tableHandle = redirection.getTableHandle();
- if (tableHandle.isEmpty()) {
+ if (redirection.getTableHandle().isEmpty()) {
throw semanticException(TABLE_NOT_FOUND, statement, "Table '%s' does not exist", tableName);
}
if (redirection.getRedirectedTableName().isPresent()) {
diff --git a/core/trino-main/src/main/java/io/trino/execution/QueryExecution.java b/core/trino-main/src/main/java/io/trino/execution/QueryExecution.java
index f0608f70b7ab..38c5a85358fa 100644
--- a/core/trino-main/src/main/java/io/trino/execution/QueryExecution.java
+++ b/core/trino-main/src/main/java/io/trino/execution/QueryExecution.java
@@ -28,6 +28,7 @@
import io.trino.sql.planner.Plan;
import java.util.List;
+import java.util.Queue;
import java.util.function.Consumer;
import static java.util.Objects.requireNonNull;
@@ -41,7 +42,7 @@ public interface QueryExecution
void addStateChangeListener(StateChangeListener stateChangeListener);
- void addOutputInfoListener(Consumer listener);
+ void setOutputInfoListener(Consumer listener);
void outputTaskFailed(TaskId taskId, Throwable failure);
@@ -86,23 +87,23 @@ interface QueryExecutionFactory
}
/**
- * Output schema and buffer URIs for query. The info will always contain column names and types. Buffer locations will always
- * contain the full location set, but may be empty. Users of this data should keep a private copy of the seen buffers to
- * handle out of order events from the listener. Once noMoreBufferLocations is set the locations will never change, and
- * it is guaranteed that all previously sent locations are contained in the buffer locations.
+ * The info will always contain column names and types.
+ * The {@code inputsQueue} is shared between {@link QueryOutputInfo} instances.
+ * It is guaranteed that no new entries will be added to {@code inputsQueue} after {@link QueryOutputInfo}
+ * with {@link #isNoMoreInputs()} {@code == true} is created.
*/
class QueryOutputInfo
{
private final List columnNames;
private final List columnTypes;
- private final List inputs;
+ private final Queue inputsQueue;
private final boolean noMoreInputs;
- public QueryOutputInfo(List columnNames, List columnTypes, List inputs, boolean noMoreInputs)
+ public QueryOutputInfo(List columnNames, List columnTypes, Queue inputsQueue, boolean noMoreInputs)
{
this.columnNames = ImmutableList.copyOf(requireNonNull(columnNames, "columnNames is null"));
this.columnTypes = ImmutableList.copyOf(requireNonNull(columnTypes, "columnTypes is null"));
- this.inputs = ImmutableList.copyOf(requireNonNull(inputs, "inputs is null"));
+ this.inputsQueue = requireNonNull(inputsQueue, "inputsQueue is null");
this.noMoreInputs = noMoreInputs;
}
@@ -116,9 +117,15 @@ public List getColumnTypes()
return columnTypes;
}
- public List getInputs()
+ public void drainInputs(Consumer consumer)
{
- return inputs;
+ while (true) {
+ ExchangeInput input = inputsQueue.poll();
+ if (input == null) {
+ break;
+ }
+ consumer.accept(input);
+ }
}
public boolean isNoMoreInputs()
diff --git a/core/trino-main/src/main/java/io/trino/execution/QueryManager.java b/core/trino-main/src/main/java/io/trino/execution/QueryManager.java
index 1543d7c5dc39..c0d3c588e389 100644
--- a/core/trino-main/src/main/java/io/trino/execution/QueryManager.java
+++ b/core/trino-main/src/main/java/io/trino/execution/QueryManager.java
@@ -33,7 +33,7 @@ public interface QueryManager
*
* @throws NoSuchElementException if query does not exist
*/
- void addOutputInfoListener(QueryId queryId, Consumer listener)
+ void setOutputInfoListener(QueryId queryId, Consumer listener)
throws NoSuchElementException;
/**
diff --git a/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java b/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java
index fdf0e81164c8..7eb106408385 100644
--- a/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java
+++ b/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java
@@ -64,8 +64,10 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@@ -284,6 +286,7 @@ static QueryStateMachine beginWithTicker(
QUERY_STATE_LOG.debug("Query %s is %s", queryStateMachine.getQueryId(), newState);
if (newState.isDone()) {
queryStateMachine.getSession().getTransactionId().ifPresent(transactionManager::trySetInactive);
+ queryStateMachine.getOutputManager().setQueryCompleted();
}
});
@@ -711,9 +714,9 @@ private QueryStats getQueryStats(Optional rootStage, List
operatorStatsSummary.build());
}
- public void addOutputInfoListener(Consumer listener)
+ public void setOutputInfoListener(Consumer listener)
{
- outputManager.addOutputInfoListener(listener);
+ outputManager.setOutputInfoListener(listener);
}
public void addOutputTaskFailureListener(TaskFailureListener listener)
@@ -1282,21 +1285,28 @@ private static QueryStats pruneQueryStats(QueryStats queryStats)
ImmutableList.of()); // Remove the operator summaries as OperatorInfo (especially DirectExchangeClientStatus) can hold onto a large amount of memory
}
+ private QueryOutputManager getOutputManager()
+ {
+ return outputManager;
+ }
+
public static class QueryOutputManager
{
private final Executor executor;
@GuardedBy("this")
- private final List> outputInfoListeners = new ArrayList<>();
+ private Optional> listener = Optional.empty();
@GuardedBy("this")
private List columnNames;
@GuardedBy("this")
private List columnTypes;
@GuardedBy("this")
- private final List inputs = new ArrayList<>();
- @GuardedBy("this")
private boolean noMoreInputs;
+ @GuardedBy("this")
+ private boolean queryCompleted;
+
+ private final Queue inputsQueue = new ConcurrentLinkedQueue<>();
@GuardedBy("this")
private final Map outputTaskFailures = new HashMap<>();
@@ -1308,16 +1318,17 @@ public QueryOutputManager(Executor executor)
this.executor = requireNonNull(executor, "executor is null");
}
- public void addOutputInfoListener(Consumer listener)
+ public void setOutputInfoListener(Consumer listener)
{
requireNonNull(listener, "listener is null");
Optional queryOutputInfo;
synchronized (this) {
- outputInfoListeners.add(listener);
+ checkState(this.listener.isEmpty(), "listener is already set");
+ this.listener = Optional.of(listener);
queryOutputInfo = getQueryOutputInfo();
}
- queryOutputInfo.ifPresent(info -> executor.execute(() -> listener.accept(info)));
+ fireStateChangedIfReady(queryOutputInfo, Optional.of(listener));
}
public void setColumns(List columnNames, List columnTypes)
@@ -1327,16 +1338,16 @@ public void setColumns(List columnNames, List columnTypes)
checkArgument(columnNames.size() == columnTypes.size(), "columnNames and columnTypes must be the same size");
Optional queryOutputInfo;
- List> outputInfoListeners;
+ Optional> listener;
synchronized (this) {
checkState(this.columnNames == null && this.columnTypes == null, "output fields already set");
this.columnNames = ImmutableList.copyOf(columnNames);
this.columnTypes = ImmutableList.copyOf(columnTypes);
queryOutputInfo = getQueryOutputInfo();
- outputInfoListeners = ImmutableList.copyOf(this.outputInfoListeners);
+ listener = this.listener;
}
- queryOutputInfo.ifPresent(info -> fireStateChanged(info, outputInfoListeners));
+ fireStateChangedIfReady(queryOutputInfo, listener);
}
public void updateInputsForQueryResults(List newInputs, boolean noMoreInputs)
@@ -1344,16 +1355,28 @@ public void updateInputsForQueryResults(List newInputs, boolean n
requireNonNull(newInputs, "newInputs is null");
Optional queryOutputInfo;
- List> outputInfoListeners;
+ Optional> listener;
synchronized (this) {
- // noMoreInputs can be set more than once
- checkState(newInputs.isEmpty() || !this.noMoreInputs, "new inputs added after no more inputs set");
- inputs.addAll(newInputs);
- this.noMoreInputs = noMoreInputs;
+ if (!queryCompleted) {
+ // noMoreInputs can be set more than once
+ checkState(newInputs.isEmpty() || !this.noMoreInputs, "new inputs added after no more inputs set");
+ inputsQueue.addAll(newInputs);
+ this.noMoreInputs = noMoreInputs;
+ }
queryOutputInfo = getQueryOutputInfo();
- outputInfoListeners = ImmutableList.copyOf(this.outputInfoListeners);
+ listener = this.listener;
+ }
+ fireStateChangedIfReady(queryOutputInfo, listener);
+ }
+
+ public synchronized void setQueryCompleted()
+ {
+ if (queryCompleted) {
+ return;
}
- queryOutputInfo.ifPresent(info -> fireStateChanged(info, outputInfoListeners));
+ queryCompleted = true;
+ inputsQueue.clear();
+ noMoreInputs = true;
}
public void addOutputTaskFailureListener(TaskFailureListener listener)
@@ -1387,14 +1410,15 @@ private synchronized Optional getQueryOutputInfo()
if (columnNames == null || columnTypes == null) {
return Optional.empty();
}
- return Optional.of(new QueryOutputInfo(columnNames, columnTypes, inputs, noMoreInputs));
+ return Optional.of(new QueryOutputInfo(columnNames, columnTypes, inputsQueue, noMoreInputs));
}
- private void fireStateChanged(QueryOutputInfo queryOutputInfo, List> outputInfoListeners)
+ private void fireStateChangedIfReady(Optional info, Optional> listener)
{
- for (Consumer outputInfoListener : outputInfoListeners) {
- executor.execute(() -> outputInfoListener.accept(queryOutputInfo));
+ if (info.isEmpty() || listener.isEmpty()) {
+ return;
}
+ executor.execute(() -> listener.get().accept(info.get()));
}
}
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/RevokeTask.java b/core/trino-main/src/main/java/io/trino/execution/RevokeTask.java
index 43276b0c8e13..39adb347b01f 100644
--- a/core/trino-main/src/main/java/io/trino/execution/RevokeTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/RevokeTask.java
@@ -19,7 +19,6 @@
import io.trino.metadata.Metadata;
import io.trino.metadata.QualifiedObjectName;
import io.trino.metadata.RedirectionAwareTableHandle;
-import io.trino.metadata.TableHandle;
import io.trino.security.AccessControl;
import io.trino.spi.connector.CatalogSchemaName;
import io.trino.spi.security.Privilege;
@@ -101,8 +100,7 @@ private void executeRevokeOnTable(Session session, Revoke statement)
{
QualifiedObjectName tableName = createQualifiedObjectName(session, statement, statement.getName());
RedirectionAwareTableHandle redirection = metadata.getRedirectionAwareTableHandle(session, tableName);
- Optional tableHandle = redirection.getTableHandle();
- if (tableHandle.isEmpty()) {
+ if (redirection.getTableHandle().isEmpty()) {
throw semanticException(TABLE_NOT_FOUND, statement, "Table '%s' does not exist", tableName);
}
if (redirection.getRedirectedTableName().isPresent()) {
diff --git a/core/trino-main/src/main/java/io/trino/execution/RollbackTask.java b/core/trino-main/src/main/java/io/trino/execution/RollbackTask.java
index 51318e9708f9..965018c050d7 100644
--- a/core/trino-main/src/main/java/io/trino/execution/RollbackTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/RollbackTask.java
@@ -55,10 +55,7 @@ public ListenableFuture execute(
WarningCollector warningCollector)
{
Session session = stateMachine.getSession();
- if (session.getTransactionId().isEmpty()) {
- throw new TrinoException(NOT_IN_TRANSACTION, "No transaction in progress");
- }
- TransactionId transactionId = session.getTransactionId().get();
+ TransactionId transactionId = session.getTransactionId().orElseThrow(() -> new TrinoException(NOT_IN_TRANSACTION, "No transaction in progress"));
stateMachine.clearTransactionId();
transactionManager.asyncAbort(transactionId);
diff --git a/core/trino-main/src/main/java/io/trino/execution/SetPropertiesTask.java b/core/trino-main/src/main/java/io/trino/execution/SetPropertiesTask.java
index 32dbde1aa781..66f935284b01 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SetPropertiesTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SetPropertiesTask.java
@@ -118,13 +118,11 @@ private void setTableProperties(SetProperties statement, QualifiedObjectName tab
throw semanticException(NOT_SUPPORTED, statement, "Cannot set properties to a view in ALTER TABLE");
}
- Optional tableHandle = plannerContext.getMetadata().getTableHandle(session, tableName);
- if (tableHandle.isEmpty()) {
- throw semanticException(TABLE_NOT_FOUND, statement, "Table does not exist: %s", tableName);
- }
+ TableHandle tableHandle = plannerContext.getMetadata().getTableHandle(session, tableName)
+ .orElseThrow(() -> semanticException(TABLE_NOT_FOUND, statement, "Table does not exist: %s", tableName));
accessControl.checkCanSetTableProperties(session.toSecurityContext(), tableName, properties);
- plannerContext.getMetadata().setTableProperties(session, tableHandle.get(), properties);
+ plannerContext.getMetadata().setTableProperties(session, tableHandle, properties);
}
private void setMaterializedViewProperties(
diff --git a/core/trino-main/src/main/java/io/trino/execution/SetTableAuthorizationTask.java b/core/trino-main/src/main/java/io/trino/execution/SetTableAuthorizationTask.java
index aaceea09beed..651642269ce7 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SetTableAuthorizationTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SetTableAuthorizationTask.java
@@ -19,7 +19,6 @@
import io.trino.metadata.Metadata;
import io.trino.metadata.QualifiedObjectName;
import io.trino.metadata.RedirectionAwareTableHandle;
-import io.trino.metadata.TableHandle;
import io.trino.security.AccessControl;
import io.trino.spi.security.TrinoPrincipal;
import io.trino.sql.tree.Expression;
@@ -71,8 +70,7 @@ public ListenableFuture execute(
getRequiredCatalogHandle(metadata, session, statement, tableName.getCatalogName());
RedirectionAwareTableHandle redirection = metadata.getRedirectionAwareTableHandle(session, tableName);
- Optional tableHandle = redirection.getTableHandle();
- if (tableHandle.isEmpty()) {
+ if (redirection.getTableHandle().isEmpty()) {
throw semanticException(TABLE_NOT_FOUND, statement, "Table '%s' does not exist", tableName);
}
if (redirection.getRedirectedTableName().isPresent()) {
diff --git a/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java b/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java
index d2b7cf250760..77cf862590d8 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java
@@ -78,10 +78,8 @@ public SplitAssignment update(SplitAssignment assignment)
newSplits,
assignment.isNoMoreSplits());
}
- else {
- // the specified assignment is older than this one
- return this;
- }
+ // the specified assignment is older than this one
+ return this;
}
private boolean isNewer(SplitAssignment assignment)
diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java b/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java
index 1f60d3b5bc82..55c88d341350 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java
@@ -598,9 +598,9 @@ public boolean isDone()
}
@Override
- public void addOutputInfoListener(Consumer listener)
+ public void setOutputInfoListener(Consumer listener)
{
- stateMachine.addOutputInfoListener(listener);
+ stateMachine.setOutputInfoListener(listener);
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java b/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java
index f04472c2f06c..248cd6a6e7b2 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java
@@ -148,11 +148,11 @@ public List getQueries()
}
@Override
- public void addOutputInfoListener(QueryId queryId, Consumer listener)
+ public void setOutputInfoListener(QueryId queryId, Consumer listener)
{
requireNonNull(listener, "listener is null");
- queryTracker.getQuery(queryId).addOutputInfoListener(listener);
+ queryTracker.getQuery(queryId).setOutputInfoListener(listener);
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTask.java b/core/trino-main/src/main/java/io/trino/execution/SqlTask.java
index 3d2819708ff6..c8e290eecc7e 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SqlTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SqlTask.java
@@ -448,13 +448,12 @@ public TaskInfo updateTask(
this::notifyStatusChanged);
taskHolderReference.compareAndSet(taskHolder, new TaskHolder(taskExecution));
needsPlan.set(false);
+ taskExecution.start();
}
}
- if (taskExecution != null) {
- taskExecution.addSplitAssignments(splitAssignments);
- taskExecution.getTaskContext().addDynamicFilter(dynamicFilterDomains);
- }
+ taskExecution.addSplitAssignments(splitAssignments);
+ taskExecution.getTaskContext().addDynamicFilter(dynamicFilterDomains);
}
catch (Error e) {
failed(e);
diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java
index 33f2bf84b6d0..a104d0a6b527 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java
@@ -16,7 +16,6 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
@@ -42,22 +41,22 @@
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.NotThreadSafe;
-import javax.annotation.concurrent.ThreadSafe;
import java.lang.ref.WeakReference;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
import java.util.Optional;
import java.util.OptionalInt;
+import java.util.Queue;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import static com.google.common.base.MoreObjects.toStringHelper;
@@ -65,7 +64,7 @@
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
-import static com.google.common.collect.ImmutableSet.toImmutableSet;
+import static com.google.common.collect.Iterables.concat;
import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.trino.SystemSessionProperties.getInitialSplitsPerNode;
import static io.trino.SystemSessionProperties.getMaxDriversPerTask;
@@ -93,14 +92,9 @@ public class SqlTaskExecution
private final SplitMonitor splitMonitor;
- private final List> drivers = new CopyOnWriteArrayList<>();
-
private final Map driverRunnerFactoriesWithSplitLifeCycle;
private final List driverRunnerFactoriesWithTaskLifeCycle;
-
- // guarded for update only
- @GuardedBy("this")
- private final ConcurrentMap unpartitionedSplitAssignments = new ConcurrentHashMap<>();
+ private final Map driverRunnerFactoriesWithRemoteSource;
@GuardedBy("this")
private long maxAcknowledgedSplit = Long.MIN_VALUE;
@@ -113,34 +107,10 @@ public class SqlTaskExecution
@GuardedBy("this")
private final Map pendingSplitsByPlanNode;
- private final Status status;
+ // number of created Drivers that haven't yet finished
+ private final AtomicLong remainingDrivers = new AtomicLong();
- static SqlTaskExecution createSqlTaskExecution(
- TaskStateMachine taskStateMachine,
- TaskContext taskContext,
- OutputBuffer outputBuffer,
- LocalExecutionPlan localExecutionPlan,
- TaskExecutor taskExecutor,
- Executor notificationExecutor,
- SplitMonitor queryMonitor)
- {
- SqlTaskExecution task = new SqlTaskExecution(
- taskStateMachine,
- taskContext,
- outputBuffer,
- localExecutionPlan,
- taskExecutor,
- queryMonitor,
- notificationExecutor);
- try (SetThreadName ignored = new SetThreadName("Task-%s", task.getTaskId())) {
- // The scheduleDriversForTaskLifeCycle method calls enqueueDriverSplitRunner, which registers a callback with access to this object.
- // The call back is accessed from another thread, so this code cannot be placed in the constructor.
- task.scheduleDriversForTaskLifeCycle();
- return task;
- }
- }
-
- private SqlTaskExecution(
+ public SqlTaskExecution(
TaskStateMachine taskStateMachine,
TaskContext taskContext,
OutputBuffer outputBuffer,
@@ -164,24 +134,24 @@ private SqlTaskExecution(
Set partitionedSources = ImmutableSet.copyOf(localExecutionPlan.getPartitionedSourceOrder());
ImmutableMap.Builder driverRunnerFactoriesWithSplitLifeCycle = ImmutableMap.builder();
ImmutableList.Builder driverRunnerFactoriesWithTaskLifeCycle = ImmutableList.builder();
+ ImmutableMap.Builder driverRunnerFactoriesWithRemoteSource = ImmutableMap.builder();
for (DriverFactory driverFactory : localExecutionPlan.getDriverFactories()) {
Optional sourceId = driverFactory.getSourceId();
if (sourceId.isPresent() && partitionedSources.contains(sourceId.get())) {
driverRunnerFactoriesWithSplitLifeCycle.put(sourceId.get(), new DriverSplitRunnerFactory(driverFactory, true));
}
else {
- driverRunnerFactoriesWithTaskLifeCycle.add(new DriverSplitRunnerFactory(driverFactory, false));
+ DriverSplitRunnerFactory runnerFactory = new DriverSplitRunnerFactory(driverFactory, false);
+ sourceId.ifPresent(planNodeId -> driverRunnerFactoriesWithRemoteSource.put(planNodeId, runnerFactory));
+ driverRunnerFactoriesWithTaskLifeCycle.add(runnerFactory);
}
}
this.driverRunnerFactoriesWithSplitLifeCycle = driverRunnerFactoriesWithSplitLifeCycle.buildOrThrow();
this.driverRunnerFactoriesWithTaskLifeCycle = driverRunnerFactoriesWithTaskLifeCycle.build();
+ this.driverRunnerFactoriesWithRemoteSource = driverRunnerFactoriesWithRemoteSource.buildOrThrow();
this.pendingSplitsByPlanNode = this.driverRunnerFactoriesWithSplitLifeCycle.keySet().stream()
.collect(toImmutableMap(identity(), ignore -> new PendingSplitsForPlanNode()));
- this.status = new Status(
- localExecutionPlan.getDriverFactories().stream()
- .map(DriverFactory::getPipelineId)
- .collect(toImmutableSet()));
sourceStartOrder = localExecutionPlan.getPartitionedSourceOrder();
checkArgument(this.driverRunnerFactoriesWithSplitLifeCycle.keySet().equals(partitionedSources),
@@ -199,6 +169,15 @@ private SqlTaskExecution(
}
}
+ public void start()
+ {
+ try (SetThreadName ignored = new SetThreadName("Task-%s", getTaskId())) {
+ // The scheduleDriversForTaskLifeCycle method calls enqueueDriverSplitRunner, which registers a callback with access to this object.
+ // The call back is accessed from another thread, so this code cannot be placed in the constructor.
+ scheduleDriversForTaskLifeCycle();
+ }
+ }
+
// this is a separate method to ensure that the `this` reference is not leaked during construction
private static TaskHandle createTaskHandle(
TaskStateMachine taskStateMachine,
@@ -241,39 +220,20 @@ public void addSplitAssignments(List splitAssignments)
try (SetThreadName ignored = new SetThreadName("Task-%s", taskId)) {
// update our record of split assignments and schedule drivers for new partitioned splits
- Map updatedUnpartitionedSources = updateSplitAssignments(splitAssignments);
-
- // tell existing drivers about the new splits; it is safe to update drivers
- // multiple times and out of order because split assignments contain full record of
- // the unpartitioned splits
- for (WeakReference driverReference : drivers) {
- Driver driver = driverReference.get();
- // the driver can be GCed due to a failure or a limit
- if (driver == null) {
- // remove the weak reference from the list to avoid a memory leak
- // NOTE: this is a concurrent safe operation on a CopyOnWriteArrayList
- drivers.remove(driverReference);
- continue;
- }
- Optional sourceId = driver.getSourceId();
- if (sourceId.isEmpty()) {
- continue;
- }
- SplitAssignment splitAssignmentUpdate = updatedUnpartitionedSources.get(sourceId.get());
- if (splitAssignmentUpdate == null) {
- continue;
- }
- driver.updateSplitAssignment(splitAssignmentUpdate);
+ Set updatedUnpartitionedSources = updateSplitAssignments(splitAssignments);
+ for (PlanNodeId planNodeId : updatedUnpartitionedSources) {
+ DriverSplitRunnerFactory factory = driverRunnerFactoriesWithRemoteSource.get(planNodeId);
+ // schedule splits outside the lock
+ factory.scheduleSplits();
}
-
// we may have transitioned to no more splits, so check for completion
checkTaskCompletion();
}
}
- private synchronized Map updateSplitAssignments(List splitAssignments)
+ private synchronized Set updateSplitAssignments(List splitAssignments)
{
- Map updatedUnpartitionedSplitAssignments = new HashMap<>();
+ ImmutableSet.Builder updatedUnpartitionedSources = ImmutableSet.builder();
// first remove any split that was already acknowledged
long currentMaxAcknowledgedSplit = this.maxAcknowledgedSplit;
@@ -292,22 +252,20 @@ private synchronized Map updateSplitAssignments(Lis
schedulePartitionedSource(assignment);
}
else {
- scheduleUnpartitionedSource(assignment, updatedUnpartitionedSplitAssignments);
+ // tell existing drivers about the new splits
+ DriverSplitRunnerFactory factory = driverRunnerFactoriesWithRemoteSource.get(assignment.getPlanNodeId());
+ factory.enqueueSplits(assignment.getSplits(), assignment.isNoMoreSplits());
+ updatedUnpartitionedSources.add(assignment.getPlanNodeId());
}
}
- for (DriverSplitRunnerFactory driverSplitRunnerFactory :
- Iterables.concat(driverRunnerFactoriesWithSplitLifeCycle.values(), driverRunnerFactoriesWithTaskLifeCycle)) {
- driverSplitRunnerFactory.closeDriverFactoryIfFullyCreated();
- }
-
// update maxAcknowledgedSplit
maxAcknowledgedSplit = splitAssignments.stream()
.flatMap(source -> source.getSplits().stream())
.mapToLong(ScheduledSplit::getSequenceId)
.max()
.orElse(maxAcknowledgedSplit);
- return updatedUnpartitionedSplitAssignments;
+ return updatedUnpartitionedSources.build();
}
@GuardedBy("this")
@@ -358,25 +316,6 @@ private synchronized void schedulePartitionedSource(SplitAssignment splitAssignm
}
}
- private synchronized void scheduleUnpartitionedSource(SplitAssignment splitAssignmentUpdate, Map updatedUnpartitionedSources)
- {
- // create new source
- SplitAssignment newSplitAssignment;
- SplitAssignment currentSplitAssignment = unpartitionedSplitAssignments.get(splitAssignmentUpdate.getPlanNodeId());
- if (currentSplitAssignment == null) {
- newSplitAssignment = splitAssignmentUpdate;
- }
- else {
- newSplitAssignment = currentSplitAssignment.update(splitAssignmentUpdate);
- }
-
- // only record new source if something changed
- if (newSplitAssignment != currentSplitAssignment) {
- unpartitionedSplitAssignments.put(splitAssignmentUpdate.getPlanNodeId(), newSplitAssignment);
- updatedUnpartitionedSources.put(splitAssignmentUpdate.getPlanNodeId(), newSplitAssignment);
- }
- }
-
private void scheduleDriversForTaskLifeCycle()
{
// This method is called at the beginning of the task.
@@ -392,6 +331,7 @@ private void scheduleDriversForTaskLifeCycle()
driverRunnerFactory.noMoreDriverRunner();
verify(driverRunnerFactory.isNoMoreDriverRunner());
}
+ checkTaskCompletion();
}
private synchronized void enqueueDriverSplitRunner(boolean forceRunSplit, List runners)
@@ -406,7 +346,7 @@ private synchronized void enqueueDriverSplitRunner(boolean forceRunSplit, List()
{
@@ -415,7 +355,7 @@ public void onSuccess(Object result)
{
try (SetThreadName ignored = new SetThreadName("Task-%s", taskId)) {
// record driver is finished
- status.decrementRemainingDriver();
+ remainingDrivers.decrementAndGet();
checkTaskCompletion();
@@ -430,7 +370,7 @@ public void onFailure(Throwable cause)
taskStateMachine.failed(cause);
// record driver is finished
- status.decrementRemainingDriver();
+ remainingDrivers.decrementAndGet();
// fire failed event with cause
splitMonitor.splitFailedEvent(taskId, getDriverStats(), cause);
@@ -458,14 +398,14 @@ private DriverStats getDriverStats()
public synchronized Set getNoMoreSplits()
{
ImmutableSet.Builder noMoreSplits = ImmutableSet.builder();
- for (Entry entry : driverRunnerFactoriesWithSplitLifeCycle.entrySet()) {
+ for (Map.Entry entry : driverRunnerFactoriesWithSplitLifeCycle.entrySet()) {
if (entry.getValue().isNoMoreDriverRunner()) {
noMoreSplits.add(entry.getKey());
}
}
- for (SplitAssignment splitAssignment : unpartitionedSplitAssignments.values()) {
- if (splitAssignment.isNoMoreSplits()) {
- noMoreSplits.add(splitAssignment.getPlanNodeId());
+ for (Map.Entry entry : driverRunnerFactoriesWithRemoteSource.entrySet()) {
+ if (entry.getValue().isNoMoreSplits()) {
+ noMoreSplits.add(entry.getKey());
}
}
return noMoreSplits.build();
@@ -477,14 +417,14 @@ private synchronized void checkTaskCompletion()
return;
}
- // are there more partition splits expected?
- for (DriverSplitRunnerFactory driverSplitRunnerFactory : driverRunnerFactoriesWithSplitLifeCycle.values()) {
- if (!driverSplitRunnerFactory.isNoMoreDriverRunner()) {
+ // are there more drivers expected?
+ for (DriverSplitRunnerFactory driverSplitRunnerFactory : concat(driverRunnerFactoriesWithTaskLifeCycle, driverRunnerFactoriesWithSplitLifeCycle.values())) {
+ if (!driverSplitRunnerFactory.isNoMoreDrivers()) {
return;
}
}
// do we still have running tasks?
- if (status.getRemainingDriver() != 0) {
+ if (remainingDrivers.get() != 0) {
return;
}
@@ -520,8 +460,7 @@ public String toString()
{
return toStringHelper(this)
.add("taskId", taskId)
- .add("remainingDrivers", status.getRemainingDriver())
- .add("unpartitionedSplitAssignments", unpartitionedSplitAssignments)
+ .add("remainingDrivers", remainingDrivers.get())
.toString();
}
@@ -595,7 +534,16 @@ private class DriverSplitRunnerFactory
{
private final DriverFactory driverFactory;
private final PipelineContext pipelineContext;
- private boolean closed;
+
+ // number of created DriverSplitRunners that haven't created underlying Driver
+ private final AtomicInteger pendingCreations = new AtomicInteger();
+ // true if no more DriverSplitRunners will be created
+ private final AtomicBoolean noMoreDriverRunner = new AtomicBoolean();
+
+ private final List> driverReferences = new CopyOnWriteArrayList<>();
+ private final Queue queuedSplits = new ConcurrentLinkedQueue<>();
+ private final AtomicLong inFlightSplits = new AtomicLong();
+ private final AtomicBoolean noMoreSplits = new AtomicBoolean();
private DriverSplitRunnerFactory(DriverFactory driverFactory, boolean partitioned)
{
@@ -607,7 +555,8 @@ private DriverSplitRunnerFactory(DriverFactory driverFactory, boolean partitione
// The former will take two arguments, and the latter will take one. This will simplify the signature quite a bit.
public DriverSplitRunner createDriverRunner(@Nullable ScheduledSplit partitionedSplit)
{
- status.incrementPendingCreation(pipelineContext.getPipelineId());
+ checkState(!noMoreDriverRunner.get(), "noMoreDriverRunner is set");
+ pendingCreations.incrementAndGet();
// create driver context immediately so the driver existence is recorded in the stats
// the number of drivers is used to balance work across nodes
long splitWeight = partitionedSplit == null ? 0 : partitionedSplit.getSplit().getSplitWeight().getRawValue();
@@ -619,51 +568,101 @@ public Driver createDriver(DriverContext driverContext, @Nullable ScheduledSplit
{
Driver driver = driverFactory.createDriver(driverContext);
- // record driver so other threads add unpartitioned sources can see the driver
- // NOTE: this MUST be done before reading unpartitionedSources, so we see a consistent view of the unpartitioned sources
- drivers.add(new WeakReference<>(driver));
-
if (partitionedSplit != null) {
// TableScanOperator requires partitioned split to be added before the first call to process
driver.updateSplitAssignment(new SplitAssignment(partitionedSplit.getPlanNodeId(), ImmutableSet.of(partitionedSplit), true));
}
- // add unpartitioned sources
- Optional sourceId = driver.getSourceId();
- if (sourceId.isPresent()) {
- SplitAssignment splitAssignment = unpartitionedSplitAssignments.get(sourceId.get());
- if (splitAssignment != null) {
- driver.updateSplitAssignment(splitAssignment);
+ pendingCreations.decrementAndGet();
+ closeDriverFactoryIfFullyCreated();
+
+ if (driverFactory.getSourceId().isPresent() && partitionedSplit == null) {
+ driverReferences.add(new WeakReference<>(driver));
+ scheduleSplits();
+ }
+
+ return driver;
+ }
+
+ public void enqueueSplits(Set splits, boolean noMoreSplits)
+ {
+ verify(driverFactory.getSourceId().isPresent(), "not a source driver");
+ verify(!this.noMoreSplits.get() || splits.isEmpty(), "cannot add splits after noMoreSplits is set");
+ queuedSplits.addAll(splits);
+ verify(!this.noMoreSplits.get() || noMoreSplits, "cannot unset noMoreSplits");
+ if (noMoreSplits) {
+ this.noMoreSplits.set(true);
+ }
+ }
+
+ public void scheduleSplits()
+ {
+ if (driverReferences.isEmpty()) {
+ return;
+ }
+
+ PlanNodeId sourceId = driverFactory.getSourceId().orElseThrow();
+ while (!queuedSplits.isEmpty()) {
+ int activeDriversCount = 0;
+ for (WeakReference driverReference : driverReferences) {
+ Driver driver = driverReference.get();
+ if (driver == null) {
+ continue;
+ }
+ activeDriversCount++;
+ inFlightSplits.incrementAndGet();
+ ScheduledSplit split = queuedSplits.poll();
+ if (split == null) {
+ inFlightSplits.decrementAndGet();
+ break;
+ }
+ driver.updateSplitAssignment(new SplitAssignment(sourceId, ImmutableSet.of(split), false));
+ inFlightSplits.decrementAndGet();
+ }
+ if (activeDriversCount == 0) {
+ break;
}
}
- status.decrementPendingCreation(pipelineContext.getPipelineId());
- closeDriverFactoryIfFullyCreated();
+ if (noMoreSplits.get() && queuedSplits.isEmpty() && inFlightSplits.get() == 0) {
+ for (WeakReference driverReference : driverReferences) {
+ Driver driver = driverReference.get();
+ if (driver != null) {
+ driver.updateSplitAssignment(new SplitAssignment(sourceId, ImmutableSet.of(), true));
+ }
+ }
+ }
+ }
- return driver;
+ public boolean isNoMoreSplits()
+ {
+ return noMoreSplits.get();
}
public void noMoreDriverRunner()
{
- status.setNoMoreDriverRunner(pipelineContext.getPipelineId());
+ noMoreDriverRunner.set(true);
closeDriverFactoryIfFullyCreated();
}
public boolean isNoMoreDriverRunner()
{
- return status.isNoMoreDriverRunners(pipelineContext.getPipelineId());
+ return noMoreDriverRunner.get();
}
public void closeDriverFactoryIfFullyCreated()
{
- if (closed) {
+ if (driverFactory.isNoMoreDrivers()) {
return;
}
- if (!isNoMoreDriverRunner() || status.getPendingCreation(pipelineContext.getPipelineId()) != 0) {
- return;
+ if (isNoMoreDriverRunner() && pendingCreations.get() == 0) {
+ driverFactory.noMoreDrivers();
}
- driverFactory.noMoreDrivers();
- closed = true;
+ }
+
+ public boolean isNoMoreDrivers()
+ {
+ return driverFactory.isNoMoreDrivers();
}
public OptionalInt getDriverInstances()
@@ -780,94 +779,4 @@ public void stateChanged(BufferState newState)
}
}
}
-
- @ThreadSafe
- private static class Status
- {
- // no more driver runner: true if no more DriverSplitRunners will be created.
- // pending creation: number of created DriverSplitRunners that haven't created underlying Driver.
- // remaining driver: number of created Drivers that haven't yet finished.
-
- @GuardedBy("this")
- private final int pipelineWithTaskLifeCycleCount;
-
- // For these 3 perX fields, they are populated lazily. If enumeration operations on the
- // map can lead to side effects, no new entries can be created after such enumeration has
- // happened. Otherwise, the order of entry creation and the enumeration operation will
- // lead to different outcome.
- @GuardedBy("this")
- private final Map perPipeline;
- @GuardedBy("this")
- int pipelinesWithNoMoreDriverRunners;
-
- @GuardedBy("this")
- private int overallRemainingDriver;
-
- public Status(Set pipelineIds)
- {
- int pipelineWithTaskLifeCycleCount = 0;
- ImmutableMap.Builder perPipeline = ImmutableMap.builder();
- for (int pipelineId : pipelineIds) {
- perPipeline.put(pipelineId, new PerPipelineStatus());
- pipelineWithTaskLifeCycleCount++;
- }
- this.pipelineWithTaskLifeCycleCount = pipelineWithTaskLifeCycleCount;
- this.perPipeline = perPipeline.buildOrThrow();
- }
-
- public synchronized void setNoMoreDriverRunner(int pipelineId)
- {
- per(pipelineId).noMoreDriverRunners = true;
- pipelinesWithNoMoreDriverRunners++;
- }
-
- public synchronized void incrementPendingCreation(int pipelineId)
- {
- per(pipelineId).pendingCreation++;
- }
-
- public synchronized void decrementPendingCreation(int pipelineId)
- {
- per(pipelineId).pendingCreation--;
- }
-
- public synchronized void incrementRemainingDriver()
- {
- checkState(!(pipelinesWithNoMoreDriverRunners == pipelineWithTaskLifeCycleCount), "Cannot increment remainingDriver. NoMoreSplits is set.");
- overallRemainingDriver++;
- }
-
- public synchronized void decrementRemainingDriver()
- {
- checkState(overallRemainingDriver > 0, "Cannot decrement remainingDriver. Value is 0.");
- overallRemainingDriver--;
- }
-
- public synchronized int getPendingCreation(int pipelineId)
- {
- return per(pipelineId).pendingCreation;
- }
-
- public synchronized int getRemainingDriver()
- {
- return overallRemainingDriver;
- }
-
- public synchronized boolean isNoMoreDriverRunners(int pipelineId)
- {
- return per(pipelineId).noMoreDriverRunners;
- }
-
- @GuardedBy("this")
- private PerPipelineStatus per(int pipelineId)
- {
- return perPipeline.get(pipelineId);
- }
- }
-
- private static class PerPipelineStatus
- {
- int pendingCreation;
- boolean noMoreDriverRunners;
- }
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java
index 735f9319ba7e..e501684f0ebf 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java
@@ -28,7 +28,6 @@
import java.util.concurrent.Executor;
import static com.google.common.base.Throwables.throwIfUnchecked;
-import static io.trino.execution.SqlTaskExecution.createSqlTaskExecution;
import static java.util.Objects.requireNonNull;
public class SqlTaskExecutionFactory
@@ -91,13 +90,13 @@ public SqlTaskExecution create(
throw new RuntimeException(e);
}
}
- return createSqlTaskExecution(
+ return new SqlTaskExecution(
taskStateMachine,
taskContext,
outputBuffer,
localExecutionPlan,
taskExecutor,
- taskNotificationExecutor,
- splitMonitor);
+ splitMonitor,
+ taskNotificationExecutor);
}
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java b/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java
index 1ac8e0e564d5..6dc8479fce2f 100644
--- a/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java
+++ b/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java
@@ -24,11 +24,13 @@
import io.trino.operator.OperatorStats;
import io.trino.operator.PipelineStats;
import io.trino.operator.TaskStats;
+import io.trino.plugin.base.metrics.TDigestHistogram;
import io.trino.spi.eventlistener.StageGcStatistics;
import io.trino.sql.planner.PlanFragment;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.sql.planner.plan.TableScanNode;
import io.trino.util.Failures;
+import io.trino.util.Optionals;
import org.joda.time.DateTime;
import javax.annotation.concurrent.ThreadSafe;
@@ -420,6 +422,7 @@ public StageInfo getStageInfo(Supplier> taskInfosSupplier)
long failedInputBlockedTime = 0;
long bufferedDataSize = 0;
+ Optional outputBufferUtilization = Optional.empty();
long outputDataSize = 0;
long failedOutputDataSize = 0;
long outputPositions = 0;
@@ -495,6 +498,7 @@ public StageInfo getStageInfo(Supplier> taskInfosSupplier)
inputBlockedTime += taskStats.getInputBlockedTime().roundTo(NANOSECONDS);
bufferedDataSize += taskInfo.getOutputBuffers().getTotalBufferedBytes();
+ outputBufferUtilization = Optionals.combine(outputBufferUtilization, taskInfo.getOutputBuffers().getUtilization(), TDigestHistogram::mergeWith);
outputDataSize += taskStats.getOutputDataSize().toBytes();
outputPositions += taskStats.getOutputPositions();
@@ -596,6 +600,7 @@ public StageInfo getStageInfo(Supplier> taskInfosSupplier)
succinctDuration(inputBlockedTime, NANOSECONDS),
succinctDuration(failedInputBlockedTime, NANOSECONDS),
succinctBytes(bufferedDataSize),
+ outputBufferUtilization,
succinctBytes(outputDataSize),
succinctBytes(failedOutputDataSize),
outputPositions,
diff --git a/core/trino-main/src/main/java/io/trino/execution/StageStats.java b/core/trino-main/src/main/java/io/trino/execution/StageStats.java
index b783283c9f71..60fa4a84bf3b 100644
--- a/core/trino-main/src/main/java/io/trino/execution/StageStats.java
+++ b/core/trino-main/src/main/java/io/trino/execution/StageStats.java
@@ -22,12 +22,14 @@
import io.airlift.units.Duration;
import io.trino.operator.BlockedReason;
import io.trino.operator.OperatorStats;
+import io.trino.plugin.base.metrics.TDigestHistogram;
import io.trino.spi.eventlistener.StageGcStatistics;
import org.joda.time.DateTime;
import javax.annotation.concurrent.Immutable;
import java.util.List;
+import java.util.Optional;
import java.util.OptionalDouble;
import java.util.Set;
@@ -96,6 +98,7 @@ public class StageStats
private final Duration failedInputBlockedTime;
private final DataSize bufferedDataSize;
+ private final Optional outputBufferUtilization;
private final DataSize outputDataSize;
private final DataSize failedOutputDataSize;
private final long outputPositions;
@@ -170,6 +173,7 @@ public StageStats(
@JsonProperty("failedInputBlockedTime") Duration failedInputBlockedTime,
@JsonProperty("bufferedDataSize") DataSize bufferedDataSize,
+ @JsonProperty("outputBufferUtilization") Optional outputBufferUtilization,
@JsonProperty("outputDataSize") DataSize outputDataSize,
@JsonProperty("failedOutputDataSize") DataSize failedOutputDataSize,
@JsonProperty("outputPositions") long outputPositions,
@@ -258,6 +262,7 @@ public StageStats(
this.failedInputBlockedTime = requireNonNull(failedInputBlockedTime, "failedInputBlockedTime is null");
this.bufferedDataSize = requireNonNull(bufferedDataSize, "bufferedDataSize is null");
+ this.outputBufferUtilization = requireNonNull(outputBufferUtilization, "outputBufferUtilization is null");
this.outputDataSize = requireNonNull(outputDataSize, "outputDataSize is null");
this.failedOutputDataSize = requireNonNull(failedOutputDataSize, "failedOutputDataSize is null");
checkArgument(outputPositions >= 0, "outputPositions is negative");
@@ -552,6 +557,12 @@ public DataSize getBufferedDataSize()
return bufferedDataSize;
}
+ @JsonProperty
+ public Optional getOutputBufferUtilization()
+ {
+ return outputBufferUtilization;
+ }
+
@JsonProperty
public DataSize getOutputDataSize()
{
diff --git a/core/trino-main/src/main/java/io/trino/execution/TaskInfo.java b/core/trino-main/src/main/java/io/trino/execution/TaskInfo.java
index 45721bba43f6..1558095b53f2 100644
--- a/core/trino-main/src/main/java/io/trino/execution/TaskInfo.java
+++ b/core/trino-main/src/main/java/io/trino/execution/TaskInfo.java
@@ -111,7 +111,7 @@ public boolean isNeedsPlan()
public TaskInfo summarize()
{
if (taskStatus.getState().isDone()) {
- return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarize(), noMoreSplits, stats.summarizeFinal(), estimatedMemory, needsPlan);
+ return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarizeFinal(), noMoreSplits, stats.summarizeFinal(), estimatedMemory, needsPlan);
}
return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarize(), noMoreSplits, stats.summarize(), estimatedMemory, needsPlan);
}
@@ -130,7 +130,7 @@ public static TaskInfo createInitialTask(TaskId taskId, URI location, String nod
return new TaskInfo(
initialTaskStatus(taskId, location, nodeId),
DateTime.now(),
- new OutputBufferInfo("UNINITIALIZED", OPEN, true, true, 0, 0, 0, 0, bufferStates),
+ new OutputBufferInfo("UNINITIALIZED", OPEN, true, true, 0, 0, 0, 0, bufferStates, Optional.empty()),
ImmutableSet.of(),
taskStats,
Optional.empty(),
diff --git a/core/trino-main/src/main/java/io/trino/execution/TruncateTableTask.java b/core/trino-main/src/main/java/io/trino/execution/TruncateTableTask.java
index 1303913033b8..f5ed8e86eb60 100644
--- a/core/trino-main/src/main/java/io/trino/execution/TruncateTableTask.java
+++ b/core/trino-main/src/main/java/io/trino/execution/TruncateTableTask.java
@@ -26,7 +26,6 @@
import javax.inject.Inject;
import java.util.List;
-import java.util.Optional;
import static com.google.common.util.concurrent.Futures.immediateFuture;
import static io.trino.metadata.MetadataUtil.createQualifiedObjectName;
@@ -72,14 +71,12 @@ public ListenableFuture execute(
throw semanticException(NOT_SUPPORTED, statement, "Cannot truncate a view");
}
- Optional tableHandle = metadata.getTableHandle(session, tableName);
- if (tableHandle.isEmpty()) {
- throw semanticException(TABLE_NOT_FOUND, statement, "Table '%s' does not exist", tableName);
- }
+ TableHandle tableHandle = metadata.getTableHandle(session, tableName)
+ .orElseThrow(() -> semanticException(TABLE_NOT_FOUND, statement, "Table '%s' does not exist", tableName));
accessControl.checkCanTruncateTable(session.toSecurityContext(), tableName);
- metadata.truncateTable(session, tableHandle.get());
+ metadata.truncateTable(session, tableHandle);
return immediateFuture(null);
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/ArbitraryOutputBuffer.java b/core/trino-main/src/main/java/io/trino/execution/buffer/ArbitraryOutputBuffer.java
index 2e7799e45ef5..d3262645c7c1 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/ArbitraryOutputBuffer.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/ArbitraryOutputBuffer.java
@@ -25,6 +25,7 @@
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.execution.buffer.SerializedPageReference.PagesReleasedListener;
import io.trino.memory.context.LocalMemoryContext;
+import io.trino.plugin.base.metrics.TDigestHistogram;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
@@ -150,7 +151,8 @@ public OutputBufferInfo getInfo()
totalBufferedPages,
totalRowsAdded.get(),
totalPagesAdded.get(),
- infos.build());
+ infos.build(),
+ Optional.of(new TDigestHistogram(memoryManager.getUtilizationHistogram())));
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/BroadcastOutputBuffer.java b/core/trino-main/src/main/java/io/trino/execution/buffer/BroadcastOutputBuffer.java
index 95dbff4a4d03..5b40b9ac447b 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/BroadcastOutputBuffer.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/BroadcastOutputBuffer.java
@@ -24,6 +24,7 @@
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.execution.buffer.SerializedPageReference.PagesReleasedListener;
import io.trino.memory.context.LocalMemoryContext;
+import io.trino.plugin.base.metrics.TDigestHistogram;
import javax.annotation.concurrent.GuardedBy;
@@ -142,7 +143,8 @@ public OutputBufferInfo getInfo()
totalPagesAdded.get(),
buffers.stream()
.map(ClientBuffer::getInfo)
- .collect(toImmutableList()));
+ .collect(toImmutableList()),
+ Optional.of(new TDigestHistogram(memoryManager.getUtilizationHistogram())));
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/LazyOutputBuffer.java b/core/trino-main/src/main/java/io/trino/execution/buffer/LazyOutputBuffer.java
index b9f26a3412fa..9f257b50499a 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/LazyOutputBuffer.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/LazyOutputBuffer.java
@@ -137,7 +137,8 @@ public OutputBufferInfo getInfo()
0,
0,
0,
- ImmutableList.of());
+ ImmutableList.of(),
+ Optional.empty());
}
return outputBuffer.getInfo();
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferInfo.java b/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferInfo.java
index f956fa150c8a..ca4371edeb24 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferInfo.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferInfo.java
@@ -16,9 +16,11 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
+import io.trino.plugin.base.metrics.TDigestHistogram;
import java.util.List;
import java.util.Objects;
+import java.util.Optional;
import static com.google.common.base.MoreObjects.toStringHelper;
@@ -33,6 +35,7 @@ public final class OutputBufferInfo
private final long totalRowsSent;
private final long totalPagesSent;
private final List buffers;
+ private final Optional utilization;
@JsonCreator
public OutputBufferInfo(
@@ -44,7 +47,8 @@ public OutputBufferInfo(
@JsonProperty("totalBufferedPages") long totalBufferedPages,
@JsonProperty("totalRowsSent") long totalRowsSent,
@JsonProperty("totalPagesSent") long totalPagesSent,
- @JsonProperty("buffers") List buffers)
+ @JsonProperty("buffers") List buffers,
+ @JsonProperty("utilization") Optional utilization)
{
this.type = type;
this.state = state;
@@ -55,6 +59,7 @@ public OutputBufferInfo(
this.totalRowsSent = totalRowsSent;
this.totalPagesSent = totalPagesSent;
this.buffers = ImmutableList.copyOf(buffers);
+ this.utilization = utilization;
}
@JsonProperty
@@ -111,9 +116,20 @@ public long getTotalPagesSent()
return totalPagesSent;
}
+ @JsonProperty
+ public Optional getUtilization()
+ {
+ return utilization;
+ }
+
public OutputBufferInfo summarize()
{
- return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of());
+ return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of(), Optional.empty());
+ }
+
+ public OutputBufferInfo summarizeFinal()
+ {
+ return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of(), utilization);
}
@Override
@@ -134,13 +150,14 @@ public boolean equals(Object o)
Objects.equals(totalRowsSent, that.totalRowsSent) &&
Objects.equals(totalPagesSent, that.totalPagesSent) &&
state == that.state &&
- Objects.equals(buffers, that.buffers);
+ Objects.equals(buffers, that.buffers) &&
+ Objects.equals(utilization, that.utilization);
}
@Override
public int hashCode()
{
- return Objects.hash(state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, buffers);
+ return Objects.hash(state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, buffers, utilization);
}
@Override
@@ -156,6 +173,7 @@ public String toString()
.add("totalRowsSent", totalRowsSent)
.add("totalPagesSent", totalPagesSent)
.add("buffers", buffers)
+ .add("bufferUtilization", utilization)
.toString();
}
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferMemoryManager.java b/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferMemoryManager.java
index b805b86780e2..4c8c0801bc25 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferMemoryManager.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/OutputBufferMemoryManager.java
@@ -15,8 +15,10 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Suppliers;
+import com.google.common.base.Ticker;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
+import io.airlift.stats.TDigest;
import io.trino.memory.context.LocalMemoryContext;
import javax.annotation.Nullable;
@@ -54,11 +56,20 @@ class OutputBufferMemoryManager
@GuardedBy("this")
private ListenableFuture blockedOnMemory = NOT_BLOCKED;
+ private final Ticker ticker = Ticker.systemTicker();
+
private final AtomicBoolean blockOnFull = new AtomicBoolean(true);
private final Supplier memoryContextSupplier;
private final Executor notificationExecutor;
+ @GuardedBy("this")
+ private final TDigest bufferUtilization = new TDigest();
+ @GuardedBy("this")
+ private long lastBufferUtilizationRecordTime;
+ @GuardedBy("this")
+ private double lastBufferUtilization;
+
public OutputBufferMemoryManager(long maxBufferedBytes, Supplier memoryContextSupplier, Executor notificationExecutor)
{
requireNonNull(memoryContextSupplier, "memoryContextSupplier is null");
@@ -66,6 +77,8 @@ public OutputBufferMemoryManager(long maxBufferedBytes, Supplier getBufferBlockedFuture()
{
if (bufferBlockedFuture == null) {
@@ -155,6 +177,13 @@ public double getUtilization()
return bufferedBytes.get() / (double) maxBufferedBytes;
}
+ public synchronized TDigest getUtilizationHistogram()
+ {
+ // always get most up to date histogram
+ recordBufferUtilization();
+ return TDigest.copyOf(bufferUtilization);
+ }
+
public boolean isOverutilized()
{
return isBufferFull();
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/PagesSerdeUtil.java b/core/trino-main/src/main/java/io/trino/execution/buffer/PagesSerdeUtil.java
index 6d45924364f0..7f0d2c199b21 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/PagesSerdeUtil.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/PagesSerdeUtil.java
@@ -130,7 +130,7 @@ protected Page computeNext()
context.close(); // Release context buffers
return endOfData();
}
- else if (read != headerBuffer.length) {
+ if (read != headerBuffer.length) {
throw new EOFException();
}
@@ -167,7 +167,7 @@ protected Slice computeNext()
if (read <= 0) {
return endOfData();
}
- else if (read != headerBuffer.length) {
+ if (read != headerBuffer.length) {
throw new EOFException();
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/PartitionedOutputBuffer.java b/core/trino-main/src/main/java/io/trino/execution/buffer/PartitionedOutputBuffer.java
index b5b3677d8735..0a7dfcac1d09 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/PartitionedOutputBuffer.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/PartitionedOutputBuffer.java
@@ -22,6 +22,7 @@
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.execution.buffer.SerializedPageReference.PagesReleasedListener;
import io.trino.memory.context.LocalMemoryContext;
+import io.trino.plugin.base.metrics.TDigestHistogram;
import java.util.List;
import java.util.Optional;
@@ -127,7 +128,8 @@ public OutputBufferInfo getInfo()
totalBufferedPages,
totalRowsAdded.get(),
totalPagesAdded.get(),
- infos.build());
+ infos.build(),
+ Optional.of(new TDigestHistogram(memoryManager.getUtilizationHistogram())));
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/buffer/SpoolingExchangeOutputBuffer.java b/core/trino-main/src/main/java/io/trino/execution/buffer/SpoolingExchangeOutputBuffer.java
index 2d5f0bb7146e..e52860fcc7dc 100644
--- a/core/trino-main/src/main/java/io/trino/execution/buffer/SpoolingExchangeOutputBuffer.java
+++ b/core/trino-main/src/main/java/io/trino/execution/buffer/SpoolingExchangeOutputBuffer.java
@@ -86,7 +86,8 @@ public OutputBufferInfo getInfo()
totalPagesAdded.get(),
totalRowsAdded.get(),
totalPagesAdded.get(),
- ImmutableList.of());
+ ImmutableList.of(),
+ Optional.empty());
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/execution/resourcegroups/InternalResourceGroup.java b/core/trino-main/src/main/java/io/trino/execution/resourcegroups/InternalResourceGroup.java
index 80bd6ab34857..05813cc68c5e 100644
--- a/core/trino-main/src/main/java/io/trino/execution/resourcegroups/InternalResourceGroup.java
+++ b/core/trino-main/src/main/java/io/trino/execution/resourcegroups/InternalResourceGroup.java
@@ -234,12 +234,10 @@ private ResourceGroupState getState()
if (canRunMore()) {
return CAN_RUN;
}
- else if (canQueueMore()) {
+ if (canQueueMore()) {
return CAN_QUEUE;
}
- else {
- return FULL;
- }
+ return FULL;
}
}
@@ -877,9 +875,7 @@ private static long getSubGroupSchedulingPriority(SchedulingPolicy policy, Inter
if (policy == QUERY_PRIORITY) {
return group.getHighestQueryPriority();
}
- else {
- return group.computeSchedulingWeight();
- }
+ return group.computeSchedulingWeight();
}
private long computeSchedulingWeight()
diff --git a/core/trino-main/src/main/java/io/trino/execution/resourcegroups/StochasticPriorityQueue.java b/core/trino-main/src/main/java/io/trino/execution/resourcegroups/StochasticPriorityQueue.java
index ae6c92b90e26..81125d0f31d1 100644
--- a/core/trino-main/src/main/java/io/trino/execution/resourcegroups/StochasticPriorityQueue.java
+++ b/core/trino-main/src/main/java/io/trino/execution/resourcegroups/StochasticPriorityQueue.java
@@ -257,9 +257,7 @@ public Node addNode(E value, long tickets)
if (left.get().descendants < right.get().descendants) {
return left.get().addNode(value, tickets);
}
- else {
- return right.get().addNode(value, tickets);
- }
+ return right.get().addNode(value, tickets);
}
Node child = new Node<>(Optional.of(this), value);
diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java
index 1a6b1542e9ed..5610763bb533 100644
--- a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java
+++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java
@@ -148,10 +148,8 @@ public ScheduleResult schedule()
if (blockedReason != null) {
return new ScheduleResult(sourceSchedulers.isEmpty(), newTasks, blocked, blockedReason, splitsScheduled);
}
- else {
- checkState(blocked.isDone(), "blockedReason not provided when scheduler is blocked");
- return new ScheduleResult(sourceSchedulers.isEmpty(), newTasks, splitsScheduled);
- }
+ checkState(blocked.isDone(), "blockedReason not provided when scheduler is blocked");
+ return new ScheduleResult(sourceSchedulers.isEmpty(), newTasks, splitsScheduled);
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java b/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java
index f197211eab0c..b82029b72062 100644
--- a/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java
+++ b/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java
@@ -528,47 +528,45 @@ private static long asArrayIndex(Object object)
}
return jsonNode.longValue();
}
- else {
- TypedValue value = (TypedValue) object;
- Type type = value.getType();
- if (type.equals(BIGINT) || type.equals(INTEGER) || type.equals(SMALLINT) || type.equals(TINYINT)) {
- return value.getLongValue();
+ TypedValue value = (TypedValue) object;
+ Type type = value.getType();
+ if (type.equals(BIGINT) || type.equals(INTEGER) || type.equals(SMALLINT) || type.equals(TINYINT)) {
+ return value.getLongValue();
+ }
+ if (type.equals(DOUBLE)) {
+ try {
+ return DoubleOperators.castToLong(value.getDoubleValue());
}
- if (type.equals(DOUBLE)) {
- try {
- return DoubleOperators.castToLong(value.getDoubleValue());
- }
- catch (Exception e) {
- throw new PathEvaluationError(e);
- }
+ catch (Exception e) {
+ throw new PathEvaluationError(e);
}
- if (type.equals(REAL)) {
- try {
- return RealOperators.castToLong(value.getLongValue());
- }
- catch (Exception e) {
- throw new PathEvaluationError(e);
- }
+ }
+ if (type.equals(REAL)) {
+ try {
+ return RealOperators.castToLong(value.getLongValue());
}
- if (type instanceof DecimalType) {
- DecimalType decimalType = (DecimalType) type;
- int precision = decimalType.getPrecision();
- int scale = decimalType.getScale();
- if (((DecimalType) type).isShort()) {
- long tenToScale = longTenToNth(DecimalConversions.intScale(scale));
- return DecimalCasts.shortDecimalToBigint(value.getLongValue(), precision, scale, tenToScale);
- }
- Int128 tenToScale = Int128Math.powerOfTen(DecimalConversions.intScale(scale));
- try {
- return DecimalCasts.longDecimalToBigint((Int128) value.getObjectValue(), precision, scale, tenToScale);
- }
- catch (Exception e) {
- throw new PathEvaluationError(e);
- }
+ catch (Exception e) {
+ throw new PathEvaluationError(e);
+ }
+ }
+ if (type instanceof DecimalType) {
+ DecimalType decimalType = (DecimalType) type;
+ int precision = decimalType.getPrecision();
+ int scale = decimalType.getScale();
+ if (((DecimalType) type).isShort()) {
+ long tenToScale = longTenToNth(DecimalConversions.intScale(scale));
+ return DecimalCasts.shortDecimalToBigint(value.getLongValue(), precision, scale, tenToScale);
+ }
+ Int128 tenToScale = Int128Math.powerOfTen(DecimalConversions.intScale(scale));
+ try {
+ return DecimalCasts.longDecimalToBigint((Int128) value.getObjectValue(), precision, scale, tenToScale);
+ }
+ catch (Exception e) {
+ throw new PathEvaluationError(e);
}
-
- throw itemTypeError("NUMBER", type.getDisplayName());
}
+
+ throw itemTypeError("NUMBER", type.getDisplayName());
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/metadata/LiteralFunction.java b/core/trino-main/src/main/java/io/trino/metadata/LiteralFunction.java
index 99d7bea38572..f519582e934c 100644
--- a/core/trino-main/src/main/java/io/trino/metadata/LiteralFunction.java
+++ b/core/trino-main/src/main/java/io/trino/metadata/LiteralFunction.java
@@ -113,9 +113,7 @@ public static Type typeForMagicLiteral(Type type)
if (type instanceof VarcharType) {
return type;
}
- else {
- return VARBINARY;
- }
+ return VARBINARY;
}
if (clazz == boolean.class) {
return BOOLEAN;
diff --git a/core/trino-main/src/main/java/io/trino/metadata/QualifiedTablePrefix.java b/core/trino-main/src/main/java/io/trino/metadata/QualifiedTablePrefix.java
index e500720ac777..3666e85c082b 100644
--- a/core/trino-main/src/main/java/io/trino/metadata/QualifiedTablePrefix.java
+++ b/core/trino-main/src/main/java/io/trino/metadata/QualifiedTablePrefix.java
@@ -100,12 +100,10 @@ public SchemaTablePrefix asSchemaTablePrefix()
if (schemaName.isEmpty()) {
return new SchemaTablePrefix();
}
- else if (tableName.isEmpty()) {
+ if (tableName.isEmpty()) {
return new SchemaTablePrefix(schemaName.get());
}
- else {
- return new SchemaTablePrefix(schemaName.get(), tableName.get());
- }
+ return new SchemaTablePrefix(schemaName.get(), tableName.get());
}
public Optional asQualifiedObjectName()
diff --git a/core/trino-main/src/main/java/io/trino/metadata/SignatureBinder.java b/core/trino-main/src/main/java/io/trino/metadata/SignatureBinder.java
index 748289c8572c..4c05c47d1451 100644
--- a/core/trino-main/src/main/java/io/trino/metadata/SignatureBinder.java
+++ b/core/trino-main/src/main/java/io/trino/metadata/SignatureBinder.java
@@ -479,11 +479,9 @@ private boolean appendConstraintSolvers(
ImmutableList.Builder formalTypeParameterTypeSignatures = ImmutableList.builder();
for (TypeSignatureParameter formalTypeParameter : formalTypeSignature.getParameters()) {
- Optional typeSignature = formalTypeParameter.getTypeSignatureOrNamedTypeSignature();
- if (typeSignature.isEmpty()) {
- throw new UnsupportedOperationException("Types with both type parameters and literal parameters at the same time are not supported");
- }
- formalTypeParameterTypeSignatures.add(typeSignature.get());
+ TypeSignature typeSignature = formalTypeParameter.getTypeSignatureOrNamedTypeSignature()
+ .orElseThrow(() -> new UnsupportedOperationException("Types with both type parameters and literal parameters at the same time are not supported"));
+ formalTypeParameterTypeSignatures.add(typeSignature);
}
return appendConstraintSolvers(
@@ -692,13 +690,11 @@ private boolean canCast(Type fromType, Type toType)
}
return true;
}
- else if (toType instanceof JsonType) {
+ if (toType instanceof JsonType) {
return fromType.getTypeParameters().stream()
.allMatch(fromTypeParameter -> canCast(fromTypeParameter, toType));
}
- else {
- return false;
- }
+ return false;
}
if (fromType instanceof JsonType) {
if (toType instanceof RowType) {
diff --git a/core/trino-main/src/main/java/io/trino/operator/DirectExchangeClient.java b/core/trino-main/src/main/java/io/trino/operator/DirectExchangeClient.java
index 32415aa1d3f2..f5e461cc3e5d 100644
--- a/core/trino-main/src/main/java/io/trino/operator/DirectExchangeClient.java
+++ b/core/trino-main/src/main/java/io/trino/operator/DirectExchangeClient.java
@@ -45,6 +45,7 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.Sets.newConcurrentHashSet;
import static java.util.Objects.requireNonNull;
@@ -155,10 +156,7 @@ public synchronized void addLocation(TaskId taskId, URI location)
return;
}
- // ignore duplicate locations
- if (allClients.containsKey(location)) {
- return;
- }
+ checkArgument(!allClients.containsKey(location), "location already exist: %s", location);
checkState(!noMoreLocations, "No more locations already set");
buffer.addTask(taskId);
diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java
index 169bdc534fe4..cd8ff2bcc31b 100644
--- a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java
+++ b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java
@@ -235,9 +235,7 @@ public CounterStat getInputDataSize()
if (inputOperator != null) {
return inputOperator.getInputDataSize();
}
- else {
- return new CounterStat();
- }
+ return new CounterStat();
}
public CounterStat getInputPositions()
@@ -246,9 +244,7 @@ public CounterStat getInputPositions()
if (inputOperator != null) {
return inputOperator.getInputPositions();
}
- else {
- return new CounterStat();
- }
+ return new CounterStat();
}
public CounterStat getOutputDataSize()
@@ -257,9 +253,7 @@ public CounterStat getOutputDataSize()
if (inputOperator != null) {
return inputOperator.getOutputDataSize();
}
- else {
- return new CounterStat();
- }
+ return new CounterStat();
}
public CounterStat getOutputPositions()
@@ -268,9 +262,7 @@ public CounterStat getOutputPositions()
if (inputOperator != null) {
return inputOperator.getOutputPositions();
}
- else {
- return new CounterStat();
- }
+ return new CounterStat();
}
public long getPhysicalWrittenDataSize()
diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java b/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java
index 812e5535f325..ba86b1998612 100644
--- a/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java
@@ -16,6 +16,8 @@
import com.google.common.collect.ImmutableList;
import io.trino.sql.planner.plan.PlanNodeId;
+import javax.annotation.concurrent.GuardedBy;
+
import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;
@@ -34,7 +36,8 @@ public class DriverFactory
private final Optional sourceId;
private final OptionalInt driverInstances;
- private boolean closed;
+ @GuardedBy("this")
+ private boolean noMoreDrivers;
public DriverFactory(int pipelineId, boolean inputDriver, boolean outputDriver, List operatorFactories, OptionalInt driverInstances)
{
@@ -91,7 +94,7 @@ public List getOperatorFactories()
public synchronized Driver createDriver(DriverContext driverContext)
{
- checkState(!closed, "DriverFactory is already closed");
+ checkState(!noMoreDrivers, "noMoreDrivers is already set");
requireNonNull(driverContext, "driverContext is null");
ImmutableList.Builder operators = ImmutableList.builder();
for (OperatorFactory operatorFactory : operatorFactories) {
@@ -103,12 +106,17 @@ public synchronized Driver createDriver(DriverContext driverContext)
public synchronized void noMoreDrivers()
{
- if (closed) {
+ if (noMoreDrivers) {
return;
}
- closed = true;
+ noMoreDrivers = true;
for (OperatorFactory operatorFactory : operatorFactories) {
operatorFactory.noMoreOperators();
}
}
+
+ public synchronized boolean isNoMoreDrivers()
+ {
+ return noMoreDrivers;
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/ExchangeOperator.java b/core/trino-main/src/main/java/io/trino/operator/ExchangeOperator.java
index 3d1cb6ef8911..d84b3b0c4c48 100644
--- a/core/trino-main/src/main/java/io/trino/operator/ExchangeOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/ExchangeOperator.java
@@ -29,6 +29,10 @@
import io.trino.spi.exchange.ExchangeId;
import io.trino.split.RemoteSplit;
import io.trino.sql.planner.plan.PlanNodeId;
+import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
+import it.unimi.dsi.fastutil.ints.IntSet;
+
+import javax.annotation.concurrent.ThreadSafe;
import java.util.Optional;
import java.util.function.Supplier;
@@ -56,6 +60,9 @@ public static class ExchangeOperatorFactory
private ExchangeDataSource exchangeDataSource;
private boolean closed;
+ private final NoMoreSplitsTracker noMoreSplitsTracker = new NoMoreSplitsTracker();
+ private int nextOperatorInstanceId;
+
public ExchangeOperatorFactory(
int operatorId,
PlanNodeId sourceId,
@@ -99,16 +106,28 @@ public SourceOperator createOperator(DriverContext driverContext)
retryPolicy,
exchangeManagerRegistry);
}
- return new ExchangeOperator(
+ int operatorInstanceId = nextOperatorInstanceId;
+ nextOperatorInstanceId++;
+ ExchangeOperator exchangeOperator = new ExchangeOperator(
operatorContext,
sourceId,
exchangeDataSource,
- serdeFactory.createPagesSerde());
+ serdeFactory.createPagesSerde(),
+ noMoreSplitsTracker,
+ operatorInstanceId);
+ noMoreSplitsTracker.operatorAdded(operatorInstanceId);
+ return exchangeOperator;
}
@Override
public void noMoreOperators()
{
+ noMoreSplitsTracker.noMoreOperators();
+ if (noMoreSplitsTracker.isNoMoreSplits()) {
+ if (exchangeDataSource != null) {
+ exchangeDataSource.noMoreInputs();
+ }
+ }
closed = true;
}
}
@@ -117,18 +136,25 @@ public void noMoreOperators()
private final PlanNodeId sourceId;
private final ExchangeDataSource exchangeDataSource;
private final PagesSerde serde;
+ private final NoMoreSplitsTracker noMoreSplitsTracker;
+ private final int operatorInstanceId;
+
private ListenableFuture isBlocked = NOT_BLOCKED;
public ExchangeOperator(
OperatorContext operatorContext,
PlanNodeId sourceId,
ExchangeDataSource exchangeDataSource,
- PagesSerde serde)
+ PagesSerde serde,
+ NoMoreSplitsTracker noMoreSplitsTracker,
+ int operatorInstanceId)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.sourceId = requireNonNull(sourceId, "sourceId is null");
this.exchangeDataSource = requireNonNull(exchangeDataSource, "exchangeDataSource is null");
this.serde = requireNonNull(serde, "serde is null");
+ this.noMoreSplitsTracker = requireNonNull(noMoreSplitsTracker, "noMoreSplitsTracker is null");
+ this.operatorInstanceId = operatorInstanceId;
operatorContext.setInfoSupplier(exchangeDataSource::getInfo);
}
@@ -154,7 +180,10 @@ public Supplier> addSplit(Split split)
@Override
public void noMoreSplits()
{
- exchangeDataSource.noMoreInputs();
+ noMoreSplitsTracker.noMoreSplits(operatorInstanceId);
+ if (noMoreSplitsTracker.isNoMoreSplits()) {
+ exchangeDataSource.noMoreInputs();
+ }
}
@Override
@@ -220,4 +249,34 @@ public void close()
{
exchangeDataSource.close();
}
+
+ @ThreadSafe
+ private static class NoMoreSplitsTracker
+ {
+ private final IntSet allOperators = new IntOpenHashSet();
+ private final IntSet noMoreSplitsOperators = new IntOpenHashSet();
+ private boolean noMoreOperators;
+
+ public synchronized void operatorAdded(int operatorInstanceId)
+ {
+ checkState(!noMoreOperators, "noMoreOperators is set");
+ allOperators.add(operatorInstanceId);
+ }
+
+ public synchronized void noMoreOperators()
+ {
+ noMoreOperators = true;
+ }
+
+ public synchronized void noMoreSplits(int operatorInstanceId)
+ {
+ checkState(allOperators.contains(operatorInstanceId), "operatorInstanceId not found: %s", operatorInstanceId);
+ noMoreSplitsOperators.add(operatorInstanceId);
+ }
+
+ public synchronized boolean isNoMoreSplits()
+ {
+ return noMoreOperators && noMoreSplitsOperators.containsAll(allOperators);
+ }
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRankAccumulator.java b/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRankAccumulator.java
index 52070cfdb0ae..f41c1f52b8ab 100644
--- a/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRankAccumulator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRankAccumulator.java
@@ -119,7 +119,7 @@ public boolean add(long groupId, RowReference rowReference)
heapInsert(groupId, newPeerGroupIndex, 1);
return true;
}
- else if (rowReference.compareTo(strategy, peekRootRowId(groupId)) < 0) {
+ if (rowReference.compareTo(strategy, peekRootRowId(groupId)) < 0) {
// Given that total number of values >= topN, we can only consider values that are less than the root (otherwise topN would be violated)
long newPeerGroupIndex = peerGroupBuffer.allocateNewNode(rowReference.allocateRowId(), UNKNOWN_INDEX);
// Rank will increase by +1 after insertion, so only need to pop if root rank is already == topN.
@@ -131,10 +131,8 @@ else if (rowReference.compareTo(strategy, peekRootRowId(groupId)) < 0) {
}
return true;
}
- else {
- // Row cannot be accepted because the total number of values >= topN, and the row is greater than the root (meaning it's rank would be at least topN+1).
- return false;
- }
+ // Row cannot be accepted because the total number of values >= topN, and the row is greater than the root (meaning it's rank would be at least topN+1).
+ return false;
}
/**
diff --git a/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRowNumberAccumulator.java b/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRowNumberAccumulator.java
index a8f60b3578e3..866891aa1fac 100644
--- a/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRowNumberAccumulator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/GroupedTopNRowNumberAccumulator.java
@@ -88,13 +88,11 @@ public boolean add(long groupId, RowReference rowReference)
heapInsert(groupId, rowReference.allocateRowId());
return true;
}
- else if (rowReference.compareTo(strategy, heapNodeBuffer.getRowId(heapRootNodeIndex)) < 0) {
+ if (rowReference.compareTo(strategy, heapNodeBuffer.getRowId(heapRootNodeIndex)) < 0) {
heapPopAndInsert(groupId, rowReference.allocateRowId(), rowIdEvictionListener);
return true;
}
- else {
- return false;
- }
+ return false;
}
/**
diff --git a/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java b/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java
index a500a160e102..291fdf67fc79 100644
--- a/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java
@@ -369,12 +369,10 @@ public boolean needsInput()
if (finishing || outputPages != null) {
return false;
}
- else if (aggregationBuilder != null && aggregationBuilder.isFull()) {
+ if (aggregationBuilder != null && aggregationBuilder.isFull()) {
return false;
}
- else {
- return unfinishedWork == null;
- }
+ return unfinishedWork == null;
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java b/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java
index f4537718e6fe..1876c2edb5fd 100644
--- a/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java
+++ b/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java
@@ -14,8 +14,12 @@
package io.trino.operator;
import io.trino.operator.join.JoinBridgeManager;
+import io.trino.operator.join.LookupJoinOperatorFactory.JoinType;
+import io.trino.operator.join.LookupSourceFactory;
+import io.trino.operator.join.unspilled.PartitionedLookupSourceFactory;
import io.trino.spi.type.Type;
import io.trino.spiller.PartitioningSpillerFactory;
+import io.trino.sql.planner.plan.JoinNode;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.type.BlockTypeOperators;
@@ -23,31 +27,32 @@
import java.util.Optional;
import java.util.OptionalInt;
+import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.FULL_OUTER;
+import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.INNER;
+import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.LOOKUP_OUTER;
+import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.PROBE_OUTER;
+import static java.util.Objects.requireNonNull;
+
public interface OperatorFactories
{
- OperatorFactory innerJoin(
+ public OperatorFactory join(
+ JoinOperatorType joinType,
int operatorId,
PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean outputSingleMatch,
- boolean waitForBuild,
+ JoinBridgeManager extends PartitionedLookupSourceFactory> lookupSourceFactory,
boolean hasFilter,
- boolean spillingEnabled,
List probeTypes,
List probeJoinChannel,
OptionalInt probeHashChannel,
Optional> probeOutputChannels,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
BlockTypeOperators blockTypeOperators);
- OperatorFactory probeOuterJoin(
+ public OperatorFactory spillingJoin(
+ JoinOperatorType joinType,
int operatorId,
PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean outputSingleMatch,
+ JoinBridgeManager extends LookupSourceFactory> lookupSourceFactory,
boolean hasFilter,
- boolean spillingEnabled,
List probeTypes,
List probeJoinChannel,
OptionalInt probeHashChannel,
@@ -56,32 +61,62 @@ OperatorFactory probeOuterJoin(
PartitioningSpillerFactory partitioningSpillerFactory,
BlockTypeOperators blockTypeOperators);
- OperatorFactory lookupOuterJoin(
- int operatorId,
- PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean waitForBuild,
- boolean hasFilter,
- boolean spillingEnabled,
- List probeTypes,
- List probeJoinChannel,
- OptionalInt probeHashChannel,
- Optional> probeOutputChannels,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
- BlockTypeOperators blockTypeOperators);
+ class JoinOperatorType
+ {
+ private final JoinType type;
+ private final boolean outputSingleMatch;
+ private final boolean waitForBuild;
- OperatorFactory fullOuterJoin(
- int operatorId,
- PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean hasFilter,
- boolean spillingEnabled,
- List probeTypes,
- List probeJoinChannel,
- OptionalInt probeHashChannel,
- Optional> probeOutputChannels,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
- BlockTypeOperators blockTypeOperators);
+ public static JoinOperatorType ofJoinNodeType(JoinNode.Type joinNodeType, boolean outputSingleMatch, boolean waitForBuild)
+ {
+ return switch (joinNodeType) {
+ case INNER -> innerJoin(outputSingleMatch, waitForBuild);
+ case LEFT -> probeOuterJoin(outputSingleMatch);
+ case RIGHT -> lookupOuterJoin(waitForBuild);
+ case FULL -> fullOuterJoin();
+ };
+ }
+
+ public static JoinOperatorType innerJoin(boolean outputSingleMatch, boolean waitForBuild)
+ {
+ return new JoinOperatorType(INNER, outputSingleMatch, waitForBuild);
+ }
+
+ public static JoinOperatorType probeOuterJoin(boolean outputSingleMatch)
+ {
+ return new JoinOperatorType(PROBE_OUTER, outputSingleMatch, false);
+ }
+
+ public static JoinOperatorType lookupOuterJoin(boolean waitForBuild)
+ {
+ return new JoinOperatorType(LOOKUP_OUTER, false, waitForBuild);
+ }
+
+ public static JoinOperatorType fullOuterJoin()
+ {
+ return new JoinOperatorType(FULL_OUTER, false, false);
+ }
+
+ private JoinOperatorType(JoinType type, boolean outputSingleMatch, boolean waitForBuild)
+ {
+ this.type = requireNonNull(type, "type is null");
+ this.outputSingleMatch = outputSingleMatch;
+ this.waitForBuild = waitForBuild;
+ }
+
+ public boolean isOutputSingleMatch()
+ {
+ return outputSingleMatch;
+ }
+
+ public boolean isWaitForBuild()
+ {
+ return waitForBuild;
+ }
+
+ public JoinType getType()
+ {
+ return type;
+ }
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/RowReferencePageManager.java b/core/trino-main/src/main/java/io/trino/operator/RowReferencePageManager.java
index cde894270e08..0e58d279b46a 100644
--- a/core/trino-main/src/main/java/io/trino/operator/RowReferencePageManager.java
+++ b/core/trino-main/src/main/java/io/trino/operator/RowReferencePageManager.java
@@ -98,10 +98,8 @@ public Page getPage(long rowId)
checkState(currentCursor != null, "No active cursor");
return currentCursor.getPage();
}
- else {
- int pageId = rowIdBuffer.getPageId(rowId);
- return pages.get(pageId).getPage();
- }
+ int pageId = rowIdBuffer.getPageId(rowId);
+ return pages.get(pageId).getPage();
}
public int getPosition(long rowId)
@@ -111,9 +109,7 @@ public int getPosition(long rowId)
// rowId for cursors only reference the single current position
return currentCursor.getCurrentPosition();
}
- else {
- return rowIdBuffer.getPosition(rowId);
- }
+ return rowIdBuffer.getPosition(rowId);
}
private static boolean isCursorRowId(long rowId)
diff --git a/core/trino-main/src/main/java/io/trino/operator/ScanFilterAndProjectOperator.java b/core/trino-main/src/main/java/io/trino/operator/ScanFilterAndProjectOperator.java
index 9225837dd7d8..bb8c5b210c8f 100644
--- a/core/trino-main/src/main/java/io/trino/operator/ScanFilterAndProjectOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/ScanFilterAndProjectOperator.java
@@ -272,10 +272,8 @@ public TransformationState> process(Split split)
cursor = ((RecordPageSource) source).getCursor();
return ofResult(processColumnSource());
}
- else {
- pageSource = source;
- return ofResult(processPageSource());
- }
+ pageSource = source;
+ return ofResult(processPageSource());
}
WorkProcessor processColumnSource()
@@ -356,14 +354,12 @@ public ProcessState process()
outputMemoryContext.setBytes(pageBuilder.getRetainedSizeInBytes());
return ProcessState.ofResult(page);
}
- else if (finished) {
+ if (finished) {
checkState(pageBuilder.isEmpty());
return ProcessState.finished();
}
- else {
- outputMemoryContext.setBytes(pageBuilder.getRetainedSizeInBytes());
- return ProcessState.yielded();
- }
+ outputMemoryContext.setBytes(pageBuilder.getRetainedSizeInBytes());
+ return ProcessState.yielded();
}
}
@@ -396,9 +392,7 @@ public ProcessState process()
if (pageSource.isFinished()) {
return ProcessState.finished();
}
- else {
- return ProcessState.yielded();
- }
+ return ProcessState.yielded();
}
recordMaterializedBytes(page, sizeInBytes -> processedBytes += sizeInBytes);
diff --git a/core/trino-main/src/main/java/io/trino/operator/TableScanWorkProcessorOperator.java b/core/trino-main/src/main/java/io/trino/operator/TableScanWorkProcessorOperator.java
index 2b80242bf056..5f5c5ee5e8b7 100644
--- a/core/trino-main/src/main/java/io/trino/operator/TableScanWorkProcessorOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/TableScanWorkProcessorOperator.java
@@ -291,9 +291,7 @@ public ProcessState process()
if (pageSource.isFinished()) {
return ProcessState.finished();
}
- else {
- return ProcessState.yielded();
- }
+ return ProcessState.yielded();
}
return ProcessState.ofResult(page);
diff --git a/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java b/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java
index b39b624cd4b4..112ffb7dc4e9 100644
--- a/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java
@@ -246,19 +246,17 @@ private static Supplier getGroupByHashSupplier(
if (partitionChannels.isEmpty()) {
return Suppliers.ofInstance(new NoChannelGroupByHash());
}
- else {
- checkArgument(expectedPositions > 0, "expectedPositions must be > 0");
- int[] channels = Ints.toArray(partitionChannels);
- return () -> createGroupByHash(
- session,
- partitionTypes,
- channels,
- hashChannel,
- expectedPositions,
- joinCompiler,
- blockTypeOperators,
- updateMemory);
- }
+ checkArgument(expectedPositions > 0, "expectedPositions must be > 0");
+ int[] channels = Ints.toArray(partitionChannels);
+ return () -> createGroupByHash(
+ session,
+ partitionTypes,
+ channels,
+ hashChannel,
+ expectedPositions,
+ joinCompiler,
+ blockTypeOperators,
+ updateMemory);
}
private static Supplier getGroupedTopNBuilderSupplier(
@@ -281,7 +279,7 @@ private static Supplier getGroupedTopNBuilderSupplier(
generateRanking,
groupByHashSupplier.get());
}
- else if (rankingType == RankingType.RANK) {
+ if (rankingType == RankingType.RANK) {
PageWithPositionComparator comparator = new SimplePageWithPositionComparator(sourceTypes, sortChannels, sortOrders, typeOperators);
PageWithPositionEqualsAndHash equalsAndHash = new SimplePageWithPositionEqualsAndHash(ImmutableList.copyOf(sourceTypes), sortChannels, blockTypeOperators);
return () -> new GroupedTopNRankBuilder(
@@ -292,12 +290,10 @@ else if (rankingType == RankingType.RANK) {
generateRanking,
groupByHashSupplier.get());
}
- else if (rankingType == RankingType.DENSE_RANK) {
+ if (rankingType == RankingType.DENSE_RANK) {
throw new UnsupportedOperationException();
}
- else {
- throw new AssertionError("Unknown ranking type: " + rankingType);
- }
+ throw new AssertionError("Unknown ranking type: " + rankingType);
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/operator/TrinoOperatorFactories.java b/core/trino-main/src/main/java/io/trino/operator/TrinoOperatorFactories.java
index a73d63b3a5d9..c154f035c18f 100644
--- a/core/trino-main/src/main/java/io/trino/operator/TrinoOperatorFactories.java
+++ b/core/trino-main/src/main/java/io/trino/operator/TrinoOperatorFactories.java
@@ -16,8 +16,8 @@
import io.trino.operator.join.JoinBridgeManager;
import io.trino.operator.join.JoinProbe.JoinProbeFactory;
import io.trino.operator.join.LookupJoinOperatorFactory;
-import io.trino.operator.join.LookupJoinOperatorFactory.JoinType;
import io.trino.operator.join.LookupSourceFactory;
+import io.trino.operator.join.unspilled.PartitionedLookupSourceFactory;
import io.trino.spi.type.Type;
import io.trino.spiller.PartitioningSpillerFactory;
import io.trino.sql.planner.plan.PlanNodeId;
@@ -29,144 +29,76 @@
import java.util.stream.IntStream;
import static com.google.common.collect.ImmutableList.toImmutableList;
-import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.FULL_OUTER;
-import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.INNER;
-import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.LOOKUP_OUTER;
-import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.PROBE_OUTER;
public class TrinoOperatorFactories
implements OperatorFactories
{
@Override
- public OperatorFactory innerJoin(
+ public OperatorFactory join(
+ JoinOperatorType joinType,
int operatorId,
PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean outputSingleMatch,
- boolean waitForBuild,
+ JoinBridgeManager extends PartitionedLookupSourceFactory> lookupSourceFactory,
boolean hasFilter,
- boolean spillingEnabled,
List probeTypes,
List probeJoinChannel,
OptionalInt probeHashChannel,
- Optional> probeOutputChannels,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
+ Optional> probeOutputChannelsOptional,
BlockTypeOperators blockTypeOperators)
{
- return createJoinOperatorFactory(
- operatorId,
- planNodeId,
- lookupSourceFactory,
- probeTypes,
- probeJoinChannel,
- probeHashChannel,
- probeOutputChannels.orElse(rangeList(probeTypes.size())),
- INNER,
- outputSingleMatch,
- waitForBuild,
- spillingEnabled,
- totalOperatorsCount,
- partitioningSpillerFactory,
- blockTypeOperators);
- }
+ List probeOutputChannels = probeOutputChannelsOptional.orElse(rangeList(probeTypes.size()));
+ List probeOutputChannelTypes = probeOutputChannels.stream()
+ .map(probeTypes::get)
+ .collect(toImmutableList());
- @Override
- public OperatorFactory probeOuterJoin(
- int operatorId,
- PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean outputSingleMatch,
- boolean hasFilter,
- boolean spillingEnabled,
- List probeTypes,
- List probeJoinChannel,
- OptionalInt probeHashChannel,
- Optional> probeOutputChannels,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
- BlockTypeOperators blockTypeOperators)
- {
- return createJoinOperatorFactory(
+ return new io.trino.operator.join.unspilled.LookupJoinOperatorFactory(
operatorId,
planNodeId,
lookupSourceFactory,
probeTypes,
+ probeOutputChannelTypes,
+ lookupSourceFactory.getBuildOutputTypes(),
+ joinType,
+ new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
+ blockTypeOperators,
probeJoinChannel,
- probeHashChannel,
- probeOutputChannels.orElse(rangeList(probeTypes.size())),
- PROBE_OUTER,
- outputSingleMatch,
- false,
- spillingEnabled,
- totalOperatorsCount,
- partitioningSpillerFactory,
- blockTypeOperators);
+ probeHashChannel);
}
@Override
- public OperatorFactory lookupOuterJoin(
+ public OperatorFactory spillingJoin(
+ JoinOperatorType joinType,
int operatorId,
PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean waitForBuild,
+ JoinBridgeManager extends LookupSourceFactory> lookupSourceFactory,
boolean hasFilter,
- boolean spillingEnabled,
List probeTypes,
List probeJoinChannel,
OptionalInt probeHashChannel,
- Optional> probeOutputChannels,
+ Optional> probeOutputChannelsOptional,
OptionalInt totalOperatorsCount,
PartitioningSpillerFactory partitioningSpillerFactory,
BlockTypeOperators blockTypeOperators)
{
- return createJoinOperatorFactory(
- operatorId,
- planNodeId,
- lookupSourceFactory,
- probeTypes,
- probeJoinChannel,
- probeHashChannel,
- probeOutputChannels.orElse(rangeList(probeTypes.size())),
- LOOKUP_OUTER,
- false,
- waitForBuild,
- spillingEnabled,
- totalOperatorsCount,
- partitioningSpillerFactory,
- blockTypeOperators);
- }
+ List probeOutputChannels = probeOutputChannelsOptional.orElse(rangeList(probeTypes.size()));
+ List probeOutputChannelTypes = probeOutputChannels.stream()
+ .map(probeTypes::get)
+ .collect(toImmutableList());
- @Override
- public OperatorFactory fullOuterJoin(
- int operatorId,
- PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactory,
- boolean hasFilter,
- boolean spillingEnabled,
- List probeTypes,
- List probeJoinChannel,
- OptionalInt probeHashChannel,
- Optional> probeOutputChannels,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
- BlockTypeOperators blockTypeOperators)
- {
- return createJoinOperatorFactory(
+ return new LookupJoinOperatorFactory(
operatorId,
planNodeId,
lookupSourceFactory,
probeTypes,
+ probeOutputChannelTypes,
+ lookupSourceFactory.getBuildOutputTypes(),
+ joinType,
+ new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
+ blockTypeOperators,
+ totalOperatorsCount,
probeJoinChannel,
probeHashChannel,
- probeOutputChannels.orElse(rangeList(probeTypes.size())),
- FULL_OUTER,
- false,
- false,
- spillingEnabled,
- totalOperatorsCount,
- partitioningSpillerFactory,
- blockTypeOperators);
+ partitioningSpillerFactory);
}
private static List rangeList(int endExclusive)
@@ -175,59 +107,4 @@ private static List rangeList(int endExclusive)
.boxed()
.collect(toImmutableList());
}
-
- private OperatorFactory createJoinOperatorFactory(
- int operatorId,
- PlanNodeId planNodeId,
- JoinBridgeManager> lookupSourceFactoryManager,
- List probeTypes,
- List probeJoinChannel,
- OptionalInt probeHashChannel,
- List probeOutputChannels,
- JoinType joinType,
- boolean outputSingleMatch,
- boolean waitForBuild,
- boolean spillingEnabled,
- OptionalInt totalOperatorsCount,
- PartitioningSpillerFactory partitioningSpillerFactory,
- BlockTypeOperators blockTypeOperators)
- {
- List probeOutputChannelTypes = probeOutputChannels.stream()
- .map(probeTypes::get)
- .collect(toImmutableList());
-
- if (spillingEnabled) {
- return new LookupJoinOperatorFactory(
- operatorId,
- planNodeId,
- (JoinBridgeManager extends LookupSourceFactory>) lookupSourceFactoryManager,
- probeTypes,
- probeOutputChannelTypes,
- lookupSourceFactoryManager.getBuildOutputTypes(),
- joinType,
- outputSingleMatch,
- waitForBuild,
- new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
- blockTypeOperators,
- totalOperatorsCount,
- probeJoinChannel,
- probeHashChannel,
- partitioningSpillerFactory);
- }
-
- return new io.trino.operator.join.unspilled.LookupJoinOperatorFactory(
- operatorId,
- planNodeId,
- (JoinBridgeManager extends io.trino.operator.join.unspilled.PartitionedLookupSourceFactory>) lookupSourceFactoryManager,
- probeTypes,
- probeOutputChannelTypes,
- lookupSourceFactoryManager.getBuildOutputTypes(),
- joinType,
- outputSingleMatch,
- waitForBuild,
- new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel),
- blockTypeOperators,
- probeJoinChannel,
- probeHashChannel);
- }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/Aggregator.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/Aggregator.java
index c746174bdcee..45543c607509 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/Aggregator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/Aggregator.java
@@ -52,9 +52,7 @@ public Type getType()
if (step.isOutputPartial()) {
return intermediateType;
}
- else {
- return finalType;
- }
+ return finalType;
}
public void processPage(Page page)
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java
index fe8f36a7d429..8a958fd0f918 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java
@@ -267,9 +267,7 @@ private static Page filter(Page page, Block mask)
if (!mask.isNull(0) && BOOLEAN.getBoolean(mask, 0)) {
return page;
}
- else {
- return page.getPositions(new int[0], 0, 0);
- }
+ return page.getPositions(new int[0], 0, 0);
}
boolean mayHaveNull = mask.mayHaveNull();
int[] ids = new int[positions];
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java
index 05f43e2b38b3..9098f325d145 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java
@@ -59,9 +59,7 @@ public Type getType()
if (step.isOutputPartial()) {
return intermediateType;
}
- else {
- return finalType;
- }
+ return finalType;
}
public void processPage(GroupByIdBlock groupIds, Page page)
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/ParametricAggregationImplementation.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/ParametricAggregationImplementation.java
index 7e3973f4c63d..f74054653d61 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/ParametricAggregationImplementation.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/ParametricAggregationImplementation.java
@@ -399,18 +399,12 @@ static AggregationParameterKind getInputParameterKind(boolean isNullable, boolea
if (isNullable) {
return NULLABLE_BLOCK_INPUT_CHANNEL;
}
- else {
- return BLOCK_INPUT_CHANNEL;
- }
+ return BLOCK_INPUT_CHANNEL;
}
- else {
- if (isNullable) {
- throw new IllegalArgumentException(methodName + " contains a parameter with @NullablePosition that is not @BlockPosition");
- }
- else {
- return INPUT_CHANNEL;
- }
+ if (isNullable) {
+ throw new IllegalArgumentException(methodName + " contains a parameter with @NullablePosition that is not @BlockPosition");
}
+ return INPUT_CHANNEL;
}
private static Annotation baseTypeAnnotation(Annotation[] annotations, String methodName)
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/ReduceAggregationFunction.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/ReduceAggregationFunction.java
index 4959ebab017b..4bb4a734b81e 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/ReduceAggregationFunction.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/ReduceAggregationFunction.java
@@ -99,7 +99,7 @@ public AggregationImplementation specialize(BoundSignature boundSignature)
.lambdaInterfaces(BinaryFunctionInterface.class, BinaryFunctionInterface.class)
.build();
}
- else if (stateType.getJavaType() == double.class) {
+ if (stateType.getJavaType() == double.class) {
return AggregationImplementation.builder()
.inputFunction(normalizeInputMethod(boundSignature, inputType, DOUBLE_STATE_INPUT_FUNCTION))
.combineFunction(DOUBLE_STATE_COMBINE_FUNCTION)
@@ -111,7 +111,7 @@ else if (stateType.getJavaType() == double.class) {
.lambdaInterfaces(BinaryFunctionInterface.class, BinaryFunctionInterface.class)
.build();
}
- else if (stateType.getJavaType() == boolean.class) {
+ if (stateType.getJavaType() == boolean.class) {
return AggregationImplementation.builder()
.inputFunction(normalizeInputMethod(boundSignature, inputType, BOOLEAN_STATE_INPUT_FUNCTION))
.combineFunction(BOOLEAN_STATE_COMBINE_FUNCTION)
@@ -123,12 +123,10 @@ else if (stateType.getJavaType() == boolean.class) {
.lambdaInterfaces(BinaryFunctionInterface.class, BinaryFunctionInterface.class)
.build();
}
- else {
- // State with Slice or Block as native container type is intentionally not supported yet,
- // as it may result in excessive JVM memory usage of remembered set.
- // See JDK-8017163.
- throw new TrinoException(NOT_SUPPORTED, format("State type not supported for %s: %s", NAME, stateType.getDisplayName()));
- }
+ // State with Slice or Block as native container type is intentionally not supported yet,
+ // as it may result in excessive JVM memory usage of remembered set.
+ // See JDK-8017163.
+ throw new TrinoException(NOT_SUPPORTED, format("State type not supported for %s: %s", NAME, stateType.getDisplayName()));
}
private static MethodHandle normalizeInputMethod(BoundSignature boundSignature, Type inputType, MethodHandle inputMethodHandle)
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/TypedSet.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/TypedSet.java
index 8287c6bf3135..f7e915d61343 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/TypedSet.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/TypedSet.java
@@ -253,9 +253,7 @@ public boolean contains(Block block, int position)
if (block.isNull(position)) {
return containsNullElement;
}
- else {
- return blockPositionByHash.getInt(getHashPositionOfElement(block, position)) != EMPTY_SLOT;
- }
+ return blockPositionByHash.getInt(getHashPositionOfElement(block, position)) != EMPTY_SLOT;
}
/**
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java
index d4cd20cd0944..ea9b40ab054d 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java
@@ -138,17 +138,15 @@ public Work> processPage(Page page)
if (groupedAggregators.isEmpty()) {
return groupByHash.addPage(page);
}
- else {
- return new TransformWork<>(
- groupByHash.getGroupIds(page),
- groupByIdBlock -> {
- for (GroupedAggregator groupedAggregator : groupedAggregators) {
- groupedAggregator.processPage(groupByIdBlock, page);
- }
- // we do not need any output from TransformWork for this case
- return null;
- });
- }
+ return new TransformWork<>(
+ groupByHash.getGroupIds(page),
+ groupByIdBlock -> {
+ for (GroupedAggregator groupedAggregator : groupedAggregators) {
+ groupedAggregator.processPage(groupByIdBlock, page);
+ }
+ // we do not need any output from TransformWork for this case
+ return null;
+ });
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java
index da7ad0f8461a..7d266fdd76c5 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java
@@ -152,9 +152,7 @@ private boolean hasPreviousSpillCompletedSuccessfully()
getFutureValue(spillInProgress);
return true;
}
- else {
- return false;
- }
+ return false;
}
@Override
@@ -208,10 +206,8 @@ public WorkProcessor buildResult()
if (shouldMergeWithMemory(getSizeInMemoryWhenUnspilling())) {
return mergeFromDiskAndMemory();
}
- else {
- getFutureValue(spillToDisk());
- return mergeFromDisk();
- }
+ getFutureValue(spillToDisk());
+ return mergeFromDisk();
}
/**
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/GroupedTypedHistogram.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/GroupedTypedHistogram.java
index 3991be7b9fb6..5a5c36a1936a 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/GroupedTypedHistogram.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/GroupedTypedHistogram.java
@@ -436,10 +436,8 @@ private boolean processEntry(long groupId, Block block, int position, long count
addNewGroup(groupId, block, position, count);
return true;
}
- else {
- valueNode.add(count);
- return false;
- }
+ valueNode.add(count);
+ return false;
}
private void addNewGroup(long groupId, Block block, int position, long count)
@@ -488,16 +486,14 @@ private BucketDataNode createBucketDataNode(long groupId, Block block, int posit
if (nodePointer == EMPTY_BUCKET) {
return new BucketDataNode(bucketId, new ValueNode(nextNodePointer), valueHash, valueAndGroupHash, nextNodePointer, true);
}
- else if (groupAndValueMatches(groupId, block, position, nodePointer, valuePositions.get(nodePointer))) {
+ if (groupAndValueMatches(groupId, block, position, nodePointer, valuePositions.get(nodePointer))) {
// value match
return new BucketDataNode(bucketId, new ValueNode(nodePointer), valueHash, valueAndGroupHash, nodePointer, false);
}
- else {
- // keep looking
- int probe = nextProbe(probeCount);
- bucketId = nextBucketId(originalBucketId, mask, probe);
- probeCount++;
- }
+ // keep looking
+ int probe = nextProbe(probeCount);
+ bucketId = nextBucketId(originalBucketId, mask, probe);
+ probeCount++;
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/ValueStore.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/ValueStore.java
index 23873939bce2..cc4170ed0cd6 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/ValueStore.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/ValueStore.java
@@ -98,15 +98,13 @@ public int addAndGetPosition(Block block, int position, long valueHash)
return valuePointer;
}
- else if (equalOperator.equal(block, position, values, valuePointer)) {
+ if (equalOperator.equal(block, position, values, valuePointer)) {
// value at position
return valuePointer;
}
- else {
- int probe = nextProbe(probeCount);
- bucketId = nextBucketId(originalBucketId, mask, probe);
- probeCount++;
- }
+ int probe = nextProbe(probeCount);
+ bucketId = nextBucketId(originalBucketId, mask, probe);
+ probeCount++;
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/state/StateCompiler.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/state/StateCompiler.java
index 72b1f109ab72..1e55902ab293 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/state/StateCompiler.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/state/StateCompiler.java
@@ -1151,10 +1151,7 @@ public Class> getType()
Type getSqlType()
{
- if (sqlType.isEmpty()) {
- throw new IllegalArgumentException("Unsupported type: " + type);
- }
- return sqlType.get();
+ return sqlType.orElseThrow(() -> new IllegalArgumentException("Unsupported type: " + type));
}
boolean isPrimitiveType()
diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java
index a88c01732a8a..a9be3416e296 100644
--- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java
+++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java
@@ -14,7 +14,6 @@
package io.trino.operator.exchange;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.VerifyException;
import com.google.common.collect.ImmutableList;
import com.google.common.primitives.Ints;
import io.airlift.slice.XxHash64;
@@ -26,7 +25,6 @@
import io.trino.operator.PartitionFunction;
import io.trino.operator.PrecomputedHashGenerator;
import io.trino.spi.Page;
-import io.trino.spi.connector.ConnectorBucketNodeMap;
import io.trino.spi.type.Type;
import io.trino.sql.planner.NodePartitioningManager;
import io.trino.sql.planner.PartitioningHandle;
@@ -226,9 +224,7 @@ private static PartitionFunction createPartitionFunction(
// The same bucket function (with the same bucket count) as for node
// partitioning must be used. This way rows within a single bucket
// will be being processed by single thread.
- int bucketCount = nodePartitioningManager.getConnectorBucketNodeMap(session, partitioning)
- .map(ConnectorBucketNodeMap::getBucketCount)
- .orElseThrow(() -> new VerifyException("No bucket node map for partitioning: " + partitioning));
+ int bucketCount = nodePartitioningManager.getBucketNodeMap(session, partitioning).getBucketCount();
int[] bucketToPartition = new int[bucketCount];
for (int bucket = 0; bucket < bucketCount; bucket++) {
// mix the bucket bits so we don't use the same bucket number used to distribute between stages
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/DefaultPageJoiner.java b/core/trino-main/src/main/java/io/trino/operator/join/DefaultPageJoiner.java
index f9f5508dc63d..b2dc1d9fb216 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/DefaultPageJoiner.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/DefaultPageJoiner.java
@@ -384,9 +384,7 @@ private long getJoinPositionWithinPartition()
if (joinPosition >= 0) {
return lookupSourceProvider.withLease(lookupSourceLease -> lookupSourceLease.getLookupSource().joinPositionWithinPartition(joinPosition));
}
- else {
- return -1;
- }
+ return -1;
}
private Page buildOutputPage()
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java
index 0190e8c7d320..5d0f901c8173 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java
@@ -19,6 +19,7 @@
import io.trino.operator.InterpretedHashGenerator;
import io.trino.operator.Operator;
import io.trino.operator.OperatorContext;
+import io.trino.operator.OperatorFactories.JoinOperatorType;
import io.trino.operator.OperatorFactory;
import io.trino.operator.PrecomputedHashGenerator;
import io.trino.operator.ProcessorContext;
@@ -80,9 +81,7 @@ public LookupJoinOperatorFactory(
List probeTypes,
List probeOutputTypes,
List buildOutputTypes,
- JoinType joinType,
- boolean outputSingleMatch,
- boolean waitForBuild,
+ JoinOperatorType joinOperatorType,
JoinProbeFactory joinProbeFactory,
BlockTypeOperators blockTypeOperators,
OptionalInt totalOperatorsCount,
@@ -94,9 +93,9 @@ public LookupJoinOperatorFactory(
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
this.probeTypes = ImmutableList.copyOf(requireNonNull(probeTypes, "probeTypes is null"));
this.buildOutputTypes = ImmutableList.copyOf(requireNonNull(buildOutputTypes, "buildOutputTypes is null"));
- this.joinType = requireNonNull(joinType, "joinType is null");
- this.outputSingleMatch = outputSingleMatch;
- this.waitForBuild = waitForBuild;
+ this.joinType = requireNonNull(joinOperatorType.getType(), "joinType is null");
+ this.outputSingleMatch = joinOperatorType.isOutputSingleMatch();
+ this.waitForBuild = joinOperatorType.isWaitForBuild();
this.joinProbeFactory = requireNonNull(joinProbeFactory, "joinProbeFactory is null");
this.joinBridgeManager = lookupSourceFactoryManager;
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java
index a26da4b8e25e..f70775367d64 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java
@@ -267,15 +267,13 @@ static NestedLoopOutputIterator createNestedLoopOutputIterator(Page probePage, P
Page outputPage = new Page(max(probePositions, buildPositions));
return new PageRepeatingIterator(outputPage, min(probePositions, buildPositions));
}
- else if (probeChannels.length == 0 && probePage.getPositionCount() <= buildPage.getPositionCount()) {
+ if (probeChannels.length == 0 && probePage.getPositionCount() <= buildPage.getPositionCount()) {
return new PageRepeatingIterator(buildPage.getColumns(buildChannels), probePage.getPositionCount());
}
- else if (buildChannels.length == 0 && buildPage.getPositionCount() <= probePage.getPositionCount()) {
+ if (buildChannels.length == 0 && buildPage.getPositionCount() <= probePage.getPositionCount()) {
return new PageRepeatingIterator(probePage.getColumns(probeChannels), buildPage.getPositionCount());
}
- else {
- return new NestedLoopPageBuilder(probePage, buildPage, probeChannels, buildChannels);
- }
+ return new NestedLoopPageBuilder(probePage, buildPage, probeChannels, buildChannels);
}
// bi-morphic parent class for the two implementations allowed. Adding a third implementation will make getOutput megamorphic and
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/PartitionedConsumption.java b/core/trino-main/src/main/java/io/trino/operator/join/PartitionedConsumption.java
index 5752a33ec60a..4047d9095d16 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/PartitionedConsumption.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/PartitionedConsumption.java
@@ -106,9 +106,7 @@ protected Partition computeNext()
if (next != null) {
return next;
}
- else {
- return endOfData();
- }
+ return endOfData();
}
};
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/PartitionedLookupSource.java b/core/trino-main/src/main/java/io/trino/operator/join/PartitionedLookupSource.java
index b0c408833e98..e53f073b868c 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/PartitionedLookupSource.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/PartitionedLookupSource.java
@@ -69,16 +69,14 @@ public OuterPositionIterator getOuterPositionIterator()
}
};
}
- else {
- return TrackingLookupSourceSupplier.nonTracking(
- () -> new PartitionedLookupSource(
- partitions.stream()
- .map(Supplier::get)
- .collect(toImmutableList()),
- hashChannelTypes,
- Optional.empty(),
- blockTypeOperators));
- }
+ return TrackingLookupSourceSupplier.nonTracking(
+ () -> new PartitionedLookupSource(
+ partitions.stream()
+ .map(Supplier::get)
+ .collect(toImmutableList()),
+ hashChannelTypes,
+ Optional.empty(),
+ blockTypeOperators));
}
private final LookupSource[] lookupSources;
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/SortedPositionLinks.java b/core/trino-main/src/main/java/io/trino/operator/join/SortedPositionLinks.java
index c57ef8df722d..1c55fd6bf997 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/SortedPositionLinks.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/SortedPositionLinks.java
@@ -84,16 +84,14 @@ public int link(int from, int to)
positionLinks.computeIfAbsent(to, key -> new IntArrayList()).add(from);
return to;
}
- else {
- // _to_ is larger so, move the chain to _from_
- IntArrayList links = positionLinks.remove(to);
- if (links == null) {
- links = new IntArrayList();
- }
- links.add(to);
- checkState(positionLinks.put(from, links) == null, "sorted links is corrupted");
- return from;
+ // _to_ is larger so, move the chain to _from_
+ IntArrayList links = positionLinks.remove(to);
+ if (links == null) {
+ links = new IntArrayList();
}
+ links.add(to);
+ checkState(positionLinks.put(from, links) == null, "sorted links is corrupted");
+ return from;
}
private boolean isNull(int position)
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/LookupJoinOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/LookupJoinOperatorFactory.java
index 881593987516..1352737f1a63 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/LookupJoinOperatorFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/LookupJoinOperatorFactory.java
@@ -19,6 +19,7 @@
import io.trino.operator.InterpretedHashGenerator;
import io.trino.operator.Operator;
import io.trino.operator.OperatorContext;
+import io.trino.operator.OperatorFactories.JoinOperatorType;
import io.trino.operator.OperatorFactory;
import io.trino.operator.PrecomputedHashGenerator;
import io.trino.operator.ProcessorContext;
@@ -72,9 +73,7 @@ public LookupJoinOperatorFactory(
List probeTypes,
List probeOutputTypes,
List buildOutputTypes,
- JoinType joinType,
- boolean outputSingleMatch,
- boolean waitForBuild,
+ JoinOperatorType joinOperatorType,
JoinProbeFactory joinProbeFactory,
BlockTypeOperators blockTypeOperators,
List probeJoinChannels,
@@ -84,9 +83,9 @@ public LookupJoinOperatorFactory(
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
this.probeTypes = ImmutableList.copyOf(requireNonNull(probeTypes, "probeTypes is null"));
this.buildOutputTypes = ImmutableList.copyOf(requireNonNull(buildOutputTypes, "buildOutputTypes is null"));
- this.joinType = requireNonNull(joinType, "joinType is null");
- this.outputSingleMatch = outputSingleMatch;
- this.waitForBuild = waitForBuild;
+ this.joinType = requireNonNull(joinOperatorType.getType(), "joinType is null");
+ this.outputSingleMatch = joinOperatorType.isOutputSingleMatch();
+ this.waitForBuild = joinOperatorType.isWaitForBuild();
this.joinProbeFactory = requireNonNull(joinProbeFactory, "joinProbeFactory is null");
this.joinBridgeManager = lookupSourceFactoryManager;
diff --git a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/PartitionedLookupSource.java b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/PartitionedLookupSource.java
index 9cbf2bffbc91..a701de4c7049 100644
--- a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/PartitionedLookupSource.java
+++ b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/PartitionedLookupSource.java
@@ -72,16 +72,14 @@ public OuterPositionIterator getOuterPositionIterator()
}
};
}
- else {
- return TrackingLookupSourceSupplier.nonTracking(
- () -> new PartitionedLookupSource(
- partitions.stream()
- .map(Supplier::get)
- .collect(toImmutableList()),
- hashChannelTypes,
- Optional.empty(),
- blockTypeOperators));
- }
+ return TrackingLookupSourceSupplier.nonTracking(
+ () -> new PartitionedLookupSource(
+ partitions.stream()
+ .map(Supplier::get)
+ .collect(toImmutableList()),
+ hashChannelTypes,
+ Optional.empty(),
+ blockTypeOperators));
}
private final LookupSource[] lookupSources;
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java
index c22ffca477eb..98cfda83d179 100644
--- a/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java
+++ b/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java
@@ -13,6 +13,7 @@
*/
package io.trino.operator.output;
+import com.google.common.annotations.VisibleForTesting;
import io.airlift.slice.Slice;
import io.airlift.slice.Slices;
import io.trino.spi.block.Block;
@@ -80,38 +81,43 @@ public void append(IntArrayList positions, Block block)
return;
}
ensurePositionCapacity(positionCount + positions.size());
- int[] positionArray = positions.elements();
- int newByteCount = 0;
- int[] lengths = new int[positions.size()];
-
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- offsets[positionCount + i + 1] = offsets[positionCount + i];
- valueIsNull[positionCount + i] = true;
- hasNullValue = true;
+ if (block instanceof VariableWidthBlock) {
+ VariableWidthBlock variableWidthBlock = (VariableWidthBlock) block;
+ int newByteCount = 0;
+ int[] lengths = new int[positions.size()];
+ int[] sourceOffsets = new int[positions.size()];
+ int[] positionArray = positions.elements();
+
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < positions.size(); i++) {
+ int position = positionArray[i];
+ int length = variableWidthBlock.getSliceLength(position);
+ lengths[i] = length;
+ sourceOffsets[i] = variableWidthBlock.getRawSliceOffset(position);
+ newByteCount += length;
+ boolean isNull = block.isNull(position);
+ valueIsNull[positionCount + i] = isNull;
+ offsets[positionCount + i + 1] = offsets[positionCount + i] + length;
+ hasNullValue |= isNull;
+ hasNonNullValue |= !isNull;
}
- else {
- int length = block.getSliceLength(position);
+ }
+ else {
+ for (int i = 0; i < positions.size(); i++) {
+ int position = positionArray[i];
+ int length = variableWidthBlock.getSliceLength(position);
lengths[i] = length;
+ sourceOffsets[i] = variableWidthBlock.getRawSliceOffset(position);
newByteCount += length;
offsets[positionCount + i + 1] = offsets[positionCount + i] + length;
- hasNonNullValue = true;
}
+ hasNonNullValue = true;
}
+ copyBytes(variableWidthBlock.getRawSlice(), lengths, sourceOffsets, positions.size(), newByteCount);
}
else {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- int length = block.getSliceLength(position);
- lengths[i] = length;
- newByteCount += length;
- offsets[positionCount + i + 1] = offsets[positionCount + i] + length;
- }
- hasNonNullValue = true;
+ appendGenericBlock(positions, block);
}
- copyBytes(block, lengths, positionArray, positions.size(), offsets, positionCount, newByteCount);
}
@Override
@@ -132,7 +138,7 @@ public void appendRle(RunLengthEncodedBlock block)
}
else {
hasNonNullValue = true;
- duplicateBytes(block.getValue(), 0, rlePositionCount);
+ duplicateBytes(block.getSlice(0, 0, block.getSliceLength(0)), rlePositionCount);
}
}
@@ -166,16 +172,20 @@ public long getSizeInBytes()
return sizeInBytes;
}
- private void copyBytes(Block block, int[] lengths, int[] positions, int count, int[] targetOffsets, int targetOffsetsIndex, int newByteCount)
+ private void copyBytes(Slice rawSlice, int[] lengths, int[] sourceOffsets, int count, int newByteCount)
{
- ensureBytesCapacity(getCurrentOffset() + newByteCount);
+ ensureExtraBytesCapacity(newByteCount);
- for (int i = 0; i < count; i++) {
- int position = positions[i];
- if (!block.isNull(position)) {
- int length = lengths[i];
- Slice slice = block.getSlice(position, 0, length);
- slice.getBytes(0, bytes, targetOffsets[targetOffsetsIndex + i], length);
+ if (rawSlice.hasByteArray()) {
+ byte[] base = rawSlice.byteArray();
+ int byteArrayOffset = rawSlice.byteArrayOffset();
+ for (int i = 0; i < count; i++) {
+ System.arraycopy(base, byteArrayOffset + sourceOffsets[i], bytes, offsets[positionCount + i], lengths[i]);
+ }
+ }
+ else {
+ for (int i = 0; i < count; i++) {
+ rawSlice.getBytes(sourceOffsets[i], bytes, offsets[positionCount + i], lengths[i]);
}
}
@@ -184,25 +194,75 @@ private void copyBytes(Block block, int[] lengths, int[] positions, int count, i
}
/**
- * Copy {@code length} bytes from {@code block}, at position {@code position} to {@code count} consecutive positions in the {@link #bytes} array.
+ * Copy all bytes from {@code slice} to {@code count} consecutive positions in the {@link #bytes} array.
*/
- private void duplicateBytes(Block block, int position, int count)
+ private void duplicateBytes(Slice slice, int count)
{
- int length = block.getSliceLength(position);
+ int length = slice.length();
int newByteCount = toIntExact((long) count * length);
int startOffset = getCurrentOffset();
- ensureBytesCapacity(startOffset + newByteCount);
+ ensureExtraBytesCapacity(newByteCount);
+
+ duplicateBytes(slice, bytes, startOffset, count);
- Slice slice = block.getSlice(position, 0, length);
+ int currentStartOffset = startOffset + length;
for (int i = 0; i < count; i++) {
- slice.getBytes(0, bytes, startOffset + (i * length), length);
- offsets[positionCount + i + 1] = startOffset + ((i + 1) * length);
+ offsets[positionCount + i + 1] = currentStartOffset;
+ currentStartOffset += length;
}
positionCount += count;
updateSize(count, newByteCount);
}
+ /**
+ * Copy {@code length} bytes from {@code slice}, starting at offset {@code sourceOffset} to {@code count} consecutive positions in the {@link #bytes} array.
+ */
+ @VisibleForTesting
+ static void duplicateBytes(Slice slice, byte[] bytes, int startOffset, int count)
+ {
+ int length = slice.length();
+ if (length == 0) {
+ // nothing to copy
+ return;
+ }
+ // copy slice to the first position
+ slice.getBytes(0, bytes, startOffset, length);
+ int totalDuplicatedBytes = count * length;
+ int duplicatedBytes = length;
+ // copy every byte copied so far, doubling the number of bytes copied on evey iteration
+ while (duplicatedBytes * 2 <= totalDuplicatedBytes) {
+ System.arraycopy(bytes, startOffset, bytes, startOffset + duplicatedBytes, duplicatedBytes);
+ duplicatedBytes = duplicatedBytes * 2;
+ }
+ // copy the leftover
+ System.arraycopy(bytes, startOffset, bytes, startOffset + duplicatedBytes, totalDuplicatedBytes - duplicatedBytes);
+ }
+
+ private void appendGenericBlock(IntArrayList positions, Block block)
+ {
+ int newByteCount = 0;
+ for (int i = 0; i < positions.size(); i++) {
+ int position = positions.getInt(i);
+ if (block.isNull(position)) {
+ offsets[positionCount + 1] = offsets[positionCount];
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ int length = block.getSliceLength(position);
+ ensureExtraBytesCapacity(length);
+ Slice slice = block.getSlice(position, 0, length);
+ slice.getBytes(0, bytes, offsets[positionCount], length);
+ offsets[positionCount + 1] = offsets[positionCount] + length;
+ hasNonNullValue = true;
+ newByteCount += length;
+ }
+ positionCount++;
+ }
+ updateSize(positions.size(), newByteCount);
+ }
+
private void reset()
{
initialEntryCount = calculateBlockResetSize(positionCount);
@@ -228,12 +288,13 @@ private void updateSize(long positionsSize, int bytesWritten)
sizeInBytes += (SIZE_OF_BYTE + SIZE_OF_INT) * positionsSize + bytesWritten;
}
- private void ensureBytesCapacity(int bytesCapacity)
+ private void ensureExtraBytesCapacity(int extraBytesCapacity)
{
- if (bytes.length < bytesCapacity) {
+ int totalBytesCapacity = getCurrentOffset() + extraBytesCapacity;
+ if (bytes.length < totalBytesCapacity) {
int newBytesLength = Math.max(bytes.length, initialBytesSize);
- if (bytesCapacity > newBytesLength) {
- newBytesLength = Math.max(bytesCapacity, calculateNewArraySize(newBytesLength));
+ if (totalBytesCapacity > newBytesLength) {
+ newBytesLength = Math.max(totalBytesCapacity, calculateNewArraySize(newBytesLength));
}
bytes = Arrays.copyOf(bytes, newBytesLength);
updateRetainedSize();
diff --git a/core/trino-main/src/main/java/io/trino/operator/project/DictionaryAwarePageProjection.java b/core/trino-main/src/main/java/io/trino/operator/project/DictionaryAwarePageProjection.java
index dd6b9ef3ceb1..e413265e509b 100644
--- a/core/trino-main/src/main/java/io/trino/operator/project/DictionaryAwarePageProjection.java
+++ b/core/trino-main/src/main/java/io/trino/operator/project/DictionaryAwarePageProjection.java
@@ -117,9 +117,7 @@ public boolean process()
if (produceLazyBlock) {
return true;
}
- else {
- return processInternal();
- }
+ return processInternal();
}
private boolean processInternal()
@@ -200,10 +198,8 @@ public Block getResult()
return result.getLoadedBlock();
});
}
- else {
- checkState(result != null, "result has not been generated");
- return result;
- }
+ checkState(result != null, "result has not been generated");
+ return result;
}
private void setupDictionaryBlockProjection()
diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/ArrayJoin.java b/core/trino-main/src/main/java/io/trino/operator/scalar/ArrayJoin.java
index 30fb04145069..9abc95f268dd 100644
--- a/core/trino-main/src/main/java/io/trino/operator/scalar/ArrayJoin.java
+++ b/core/trino-main/src/main/java/io/trino/operator/scalar/ArrayJoin.java
@@ -166,27 +166,25 @@ private static ChoicesSpecializedSqlScalarFunction specializeArrayJoin(
methodHandle.bindTo(null),
Optional.of(STATE_FACTORY));
}
- else {
- try {
- InvocationConvention convention = new InvocationConvention(ImmutableList.of(BLOCK_POSITION), NULLABLE_RETURN, true, false);
- MethodHandle cast = functionDependencies.getCastImplementation(type, VARCHAR, convention).getMethodHandle();
-
- // if the cast doesn't take a ConnectorSession, create an adapter that drops the provided session
- if (cast.type().parameterArray()[0] != ConnectorSession.class) {
- cast = MethodHandles.dropArguments(cast, 0, ConnectorSession.class);
- }
+ try {
+ InvocationConvention convention = new InvocationConvention(ImmutableList.of(BLOCK_POSITION), NULLABLE_RETURN, true, false);
+ MethodHandle cast = functionDependencies.getCastImplementation(type, VARCHAR, convention).getMethodHandle();
- MethodHandle target = MethodHandles.insertArguments(methodHandle, 0, cast);
- return new ChoicesSpecializedSqlScalarFunction(
- boundSignature,
- FAIL_ON_NULL,
- argumentConventions,
- target,
- Optional.of(STATE_FACTORY));
- }
- catch (TrinoException e) {
- throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Input type %s not supported", type), e);
+ // if the cast doesn't take a ConnectorSession, create an adapter that drops the provided session
+ if (cast.type().parameterArray()[0] != ConnectorSession.class) {
+ cast = MethodHandles.dropArguments(cast, 0, ConnectorSession.class);
}
+
+ MethodHandle target = MethodHandles.insertArguments(methodHandle, 0, cast);
+ return new ChoicesSpecializedSqlScalarFunction(
+ boundSignature,
+ FAIL_ON_NULL,
+ argumentConventions,
+ target,
+ Optional.of(STATE_FACTORY));
+ }
+ catch (TrinoException e) {
+ throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Input type %s not supported", type), e);
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/BitwiseFunctions.java b/core/trino-main/src/main/java/io/trino/operator/scalar/BitwiseFunctions.java
index 0f861922f2d2..d86e3ce0ba61 100644
--- a/core/trino-main/src/main/java/io/trino/operator/scalar/BitwiseFunctions.java
+++ b/core/trino-main/src/main/java/io/trino/operator/scalar/BitwiseFunctions.java
@@ -202,9 +202,7 @@ public static long bitwiseRightShiftArithmeticTinyint(@SqlType(StandardTypes.TIN
if (value >= 0) {
return 0L;
}
- else {
- return -1L;
- }
+ return -1L;
}
return preserveSign(value, TINYINT_MASK, TINYINT_SIGNED_BIT) >> shift;
}
@@ -218,9 +216,7 @@ public static long bitwiseRightShiftArithmeticSmallint(@SqlType(StandardTypes.SM
if (value >= 0) {
return 0L;
}
- else {
- return -1L;
- }
+ return -1L;
}
return preserveSign(value, SMALLINT_MASK, SMALLINT_SIGNED_BIT) >> shift;
}
@@ -234,9 +230,7 @@ public static long bitwiseRightShiftArithmeticInteger(@SqlType(StandardTypes.INT
if (value >= 0) {
return 0L;
}
- else {
- return -1L;
- }
+ return -1L;
}
return preserveSign(value, INTEGER_MASK, INTEGER_SIGNED_BIT) >> shift;
}
@@ -250,9 +244,7 @@ public static long bitwiseRightShiftArithmeticBigint(@SqlType(StandardTypes.BIGI
if (value >= 0) {
return 0L;
}
- else {
- return -1L;
- }
+ return -1L;
}
return value >> shift;
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/CharacterStringCasts.java b/core/trino-main/src/main/java/io/trino/operator/scalar/CharacterStringCasts.java
index 5fcddee99505..8cf6809ea93e 100644
--- a/core/trino-main/src/main/java/io/trino/operator/scalar/CharacterStringCasts.java
+++ b/core/trino-main/src/main/java/io/trino/operator/scalar/CharacterStringCasts.java
@@ -46,9 +46,7 @@ public static Slice varcharToVarcharCast(@LiteralParameter("x") Long x, @Literal
if (x > y) {
return truncateToLength(slice, y.intValue());
}
- else {
- return slice;
- }
+ return slice;
}
@ScalarOperator(OperatorType.CAST)
@@ -59,9 +57,7 @@ public static Slice charToCharCast(@LiteralParameter("x") Long x, @LiteralParame
if (x > y) {
return truncateToLength(slice, y.intValue());
}
- else {
- return slice;
- }
+ return slice;
}
@ScalarOperator(OperatorType.CAST)
diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/ConcatWsFunction.java b/core/trino-main/src/main/java/io/trino/operator/scalar/ConcatWsFunction.java
index 2b87b118580d..f81c688d2693 100644
--- a/core/trino-main/src/main/java/io/trino/operator/scalar/ConcatWsFunction.java
+++ b/core/trino-main/src/main/java/io/trino/operator/scalar/ConcatWsFunction.java
@@ -74,10 +74,8 @@ public Slice getElement(int i)
if (elements.isNull(i)) {
return null;
}
- else {
- int sliceLength = elements.getSliceLength(i);
- return elements.getSlice(i, 0, sliceLength);
- }
+ int sliceLength = elements.getSliceLength(i);
+ return elements.getSlice(i, 0, sliceLength);
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/JoniRegexpFunctions.java b/core/trino-main/src/main/java/io/trino/operator/scalar/JoniRegexpFunctions.java
index 3a943da09600..2eabd99b2587 100644
--- a/core/trino-main/src/main/java/io/trino/operator/scalar/JoniRegexpFunctions.java
+++ b/core/trino-main/src/main/java/io/trino/operator/scalar/JoniRegexpFunctions.java
@@ -74,14 +74,10 @@ private static int getNextStart(Slice source, Matcher matcher)
if (matcher.getBegin() < source.length()) {
return matcher.getEnd() + lengthOfCodePointFromStartByte(source.getByte(matcher.getBegin()));
}
- else {
- // last match is empty and we matched end of source, move past the source length to terminate the loop
- return matcher.getEnd() + 1;
- }
- }
- else {
- return matcher.getEnd();
+ // last match is empty and we matched end of source, move past the source length to terminate the loop
+ return matcher.getEnd() + 1;
}
+ return matcher.getEnd();
}
@Description("Removes substrings matching a regular expression")
diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/MapToMapCast.java b/core/trino-main/src/main/java/io/trino/operator/scalar/MapToMapCast.java
index 54ae82504367..ece2aae99f0b 100644
--- a/core/trino-main/src/main/java/io/trino/operator/scalar/MapToMapCast.java
+++ b/core/trino-main/src/main/java/io/trino/operator/scalar/MapToMapCast.java
@@ -179,21 +179,19 @@ private MethodHandle nullChecker(Class> javaType)
if (javaType == Long.class) {
return CHECK_LONG_IS_NOT_NULL;
}
- else if (javaType == Double.class) {
+ if (javaType == Double.class) {
return CHECK_DOUBLE_IS_NOT_NULL;
}
- else if (javaType == Boolean.class) {
+ if (javaType == Boolean.class) {
return CHECK_BOOLEAN_IS_NOT_NULL;
}
- else if (javaType == Slice.class) {
+ if (javaType == Slice.class) {
return CHECK_SLICE_IS_NOT_NULL;
}
- else if (javaType == Block.class) {
+ if (javaType == Block.class) {
return CHECK_BLOCK_IS_NOT_NULL;
}
- else {
- throw new IllegalArgumentException("Unknown java type " + javaType);
- }
+ throw new IllegalArgumentException("Unknown java type " + javaType);
}
@UsedByGeneratedCode
diff --git a/core/trino-main/src/main/java/io/trino/security/AccessControl.java b/core/trino-main/src/main/java/io/trino/security/AccessControl.java
index c266ba9373a0..6b53d9320718 100644
--- a/core/trino-main/src/main/java/io/trino/security/AccessControl.java
+++ b/core/trino-main/src/main/java/io/trino/security/AccessControl.java
@@ -379,6 +379,13 @@ default void checkCanSetViewAuthorization(SecurityContext context, QualifiedObje
*/
void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, String functionName, Identity grantee, boolean grantOption);
+ /**
+ * Check if identity is allowed to create a view that executes the function.
+ *
+ * @throws AccessDeniedException if not allowed
+ */
+ void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, FunctionKind functionKind, QualifiedObjectName functionName, Identity grantee, boolean grantOption);
+
/**
* Check if identity is allowed to grant a privilege to the grantee on the specified schema.
*
diff --git a/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java b/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java
index 9946337b0ae1..40419aa91208 100644
--- a/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java
+++ b/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java
@@ -843,6 +843,21 @@ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext securityContex
grantOption));
}
+ @Override
+ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext securityContext, FunctionKind functionKind, QualifiedObjectName functionName, Identity grantee, boolean grantOption)
+ {
+ requireNonNull(securityContext, "securityContext is null");
+ requireNonNull(functionKind, "functionKind is null");
+ requireNonNull(functionName, "functionName is null");
+
+ systemAuthorizationCheck(control -> control.checkCanGrantExecuteFunctionPrivilege(
+ securityContext.toSystemSecurityContext(),
+ functionKind,
+ functionName.asCatalogSchemaRoutineName(),
+ new TrinoPrincipal(PrincipalType.USER, grantee.getUser()),
+ grantOption));
+ }
+
@Override
public void checkCanGrantSchemaPrivilege(SecurityContext securityContext, Privilege privilege, CatalogSchemaName schemaName, TrinoPrincipal grantee, boolean grantOption)
{
diff --git a/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java b/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java
index f96bcf927300..a3e34dddb459 100644
--- a/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java
+++ b/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java
@@ -271,6 +271,11 @@ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, Strin
{
}
+ @Override
+ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, FunctionKind functionKind, QualifiedObjectName functionName, Identity grantee, boolean grantOption)
+ {
+ }
+
@Override
public void checkCanGrantSchemaPrivilege(SecurityContext context, Privilege privilege, CatalogSchemaName schemaName, TrinoPrincipal grantee, boolean grantOption)
{
diff --git a/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java b/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java
index bb5fbb7e2d8a..9243fe23347b 100644
--- a/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java
+++ b/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java
@@ -379,6 +379,12 @@ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, Strin
denyGrantExecuteFunctionPrivilege(functionName, context.getIdentity(), grantee);
}
+ @Override
+ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, FunctionKind functionKind, QualifiedObjectName functionName, Identity grantee, boolean grantOption)
+ {
+ denyGrantExecuteFunctionPrivilege(functionName.toString(), context.getIdentity(), grantee);
+ }
+
@Override
public void checkCanGrantSchemaPrivilege(SecurityContext context, Privilege privilege, CatalogSchemaName schemaName, TrinoPrincipal grantee, boolean grantOption)
{
diff --git a/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java b/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java
index e86f97dd409b..ba1198188207 100644
--- a/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java
+++ b/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java
@@ -335,6 +335,12 @@ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, Strin
delegate().checkCanGrantExecuteFunctionPrivilege(context, functionName, grantee, grantOption);
}
+ @Override
+ public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, FunctionKind functionKind, QualifiedObjectName functionName, Identity grantee, boolean grantOption)
+ {
+ delegate().checkCanGrantExecuteFunctionPrivilege(context, functionKind, functionName, grantee, grantOption);
+ }
+
@Override
public void checkCanGrantSchemaPrivilege(SecurityContext context, Privilege privilege, CatalogSchemaName schemaName, TrinoPrincipal grantee, boolean grantOption)
{
diff --git a/core/trino-main/src/main/java/io/trino/security/ViewAccessControl.java b/core/trino-main/src/main/java/io/trino/security/ViewAccessControl.java
index 62a041df5445..f2b13cbf0c65 100644
--- a/core/trino-main/src/main/java/io/trino/security/ViewAccessControl.java
+++ b/core/trino-main/src/main/java/io/trino/security/ViewAccessControl.java
@@ -15,6 +15,7 @@
import io.trino.metadata.QualifiedObjectName;
import io.trino.spi.connector.CatalogSchemaTableName;
+import io.trino.spi.function.FunctionKind;
import io.trino.spi.security.AccessDeniedException;
import io.trino.spi.security.Identity;
import io.trino.spi.security.ViewExpression;
@@ -72,6 +73,12 @@ public void checkCanExecuteFunction(SecurityContext context, String functionName
wrapAccessDeniedException(() -> delegate.checkCanGrantExecuteFunctionPrivilege(context, functionName, invoker, false));
}
+ @Override
+ public void checkCanExecuteFunction(SecurityContext context, FunctionKind functionKind, QualifiedObjectName functionName)
+ {
+ wrapAccessDeniedException(() -> delegate.checkCanGrantExecuteFunctionPrivilege(context, functionKind, functionName, invoker, false));
+ }
+
@Override
public void checkCanGrantExecuteFunctionPrivilege(SecurityContext context, String functionName, Identity grantee, boolean grantOption)
{
diff --git a/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java b/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java
index cd4a7d9ca8bc..78f8e31d5446 100644
--- a/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java
+++ b/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java
@@ -108,9 +108,7 @@ public NodeState getServerState()
if (shutdownHandler.isShutdownRequested()) {
return SHUTTING_DOWN;
}
- else {
- return ACTIVE;
- }
+ return ACTIVE;
}
@ResourceSecurity(PUBLIC)
diff --git a/core/trino-main/src/main/java/io/trino/server/protocol/Query.java b/core/trino-main/src/main/java/io/trino/server/protocol/Query.java
index 6406a12e047c..c107956ae9ca 100644
--- a/core/trino-main/src/main/java/io/trino/server/protocol/Query.java
+++ b/core/trino-main/src/main/java/io/trino/server/protocol/Query.java
@@ -187,7 +187,7 @@ public static Query create(
Query result = new Query(session, slug, queryManager, queryInfoUrl, exchangeDataSource, dataProcessorExecutor, timeoutExecutor, blockEncodingSerde);
- result.queryManager.addOutputInfoListener(result.getQueryId(), result::setQueryOutputInfo);
+ result.queryManager.setOutputInfoListener(result.getQueryId(), result::setQueryOutputInfo);
result.queryManager.addStateChangeListener(result.getQueryId(), state -> {
// Wait for the query info to become available and close the exchange client if there is no output stage for the query results to be pulled from.
@@ -582,7 +582,7 @@ private synchronized void setQueryOutputInfo(QueryExecution.QueryOutputInfo outp
types = outputInfo.getColumnTypes();
}
- outputInfo.getInputs().forEach(exchangeDataSource::addInput);
+ outputInfo.drainInputs(exchangeDataSource::addInput);
if (outputInfo.isNoMoreInputs()) {
exchangeDataSource.noMoreInputs();
}
diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java
index 50d9006026e1..93db6b5a9f39 100644
--- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java
+++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java
@@ -129,10 +129,7 @@ public final class HttpRemoteTask
// The version of dynamic filters that has been successfully sent to the worker
private final AtomicLong sentDynamicFiltersVersion = new AtomicLong(INITIAL_DYNAMIC_FILTERS_VERSION);
- @GuardedBy("pendingRequestsCounter")
- private Future> currentRequest;
- @GuardedBy("pendingRequestsCounter")
- private long currentRequestStartNanos;
+ private final AtomicReference> currentRequest = new AtomicReference<>();
@GuardedBy("this")
private final SetMultimap pendingSplits = HashMultimap.create();
@@ -165,7 +162,7 @@ public final class HttpRemoteTask
private final RequestErrorTracker updateErrorTracker;
- private final AtomicInteger pendingRequestsCounter = new AtomicInteger(1);
+ private final AtomicInteger pendingRequestsCounter = new AtomicInteger(0);
private final AtomicBoolean sendPlan = new AtomicBoolean(true);
private final PartitionedSplitCountTracker partitionedSplitCountTracker;
@@ -352,7 +349,7 @@ public void start()
try (SetThreadName ignored = new SetThreadName("HttpRemoteTask-%s", taskId)) {
// to start we just need to trigger an update
started.set(true);
- scheduleUpdate();
+ triggerUpdate();
dynamicFiltersFetcher.start();
taskStatusFetcher.start();
@@ -578,6 +575,10 @@ private void scheduleUpdate()
private void triggerUpdate()
{
+ if (!started.get()) {
+ // task has not started yet
+ return;
+ }
if (pendingRequestsCounter.getAndIncrement() == 0) {
// schedule update if this is the first update requested
scheduleUpdate();
@@ -586,73 +587,59 @@ private void triggerUpdate()
private void sendUpdate()
{
- synchronized (pendingRequestsCounter) {
- TaskStatus taskStatus = getTaskStatus();
- // don't update if the task hasn't been started yet or if it is already finished
- if (!started.get() || taskStatus.getState().isDone()) {
- return;
- }
-
- int currentPendingRequestsCounter = pendingRequestsCounter.get();
- if (currentPendingRequestsCounter == 0) {
- return;
- }
+ TaskStatus taskStatus = getTaskStatus();
+ // don't update if the task is already finished
+ if (taskStatus.getState().isDone()) {
+ return;
+ }
+ checkState(started.get());
- // if there is a request already running, wait for it to complete
- // currentRequest is always cleared when request is complete
- if (currentRequest != null) {
- return;
- }
+ int currentPendingRequestsCounter = pendingRequestsCounter.get();
+ checkState(currentPendingRequestsCounter > 0, "sendUpdate shouldn't be called without pending requests");
- // if throttled due to error, asynchronously wait for timeout and try again
- ListenableFuture errorRateLimit = updateErrorTracker.acquireRequestPermit();
- if (!errorRateLimit.isDone()) {
- errorRateLimit.addListener(this::sendUpdate, executor);
- return;
- }
+ // if throttled due to error, asynchronously wait for timeout and try again
+ ListenableFuture errorRateLimit = updateErrorTracker.acquireRequestPermit();
+ if (!errorRateLimit.isDone()) {
+ errorRateLimit.addListener(this::sendUpdate, executor);
+ return;
+ }
- List splitAssignments = getSplitAssignments();
- VersionedDynamicFilterDomains dynamicFilterDomains = outboundDynamicFiltersCollector.acknowledgeAndGetNewDomains(sentDynamicFiltersVersion.get());
-
- // Workers don't need the embedded JSON representation when the fragment is sent
- Optional fragment = sendPlan.get() ? Optional.of(planFragment.withoutEmbeddedJsonRepresentation()) : Optional.empty();
- TaskUpdateRequest updateRequest = new TaskUpdateRequest(
- session.toSessionRepresentation(),
- session.getIdentity().getExtraCredentials(),
- fragment,
- splitAssignments,
- outputBuffers.get(),
- dynamicFilterDomains.getDynamicFilterDomains());
- byte[] taskUpdateRequestJson = taskUpdateRequestCodec.toJsonBytes(updateRequest);
- if (fragment.isPresent()) {
- stats.updateWithPlanBytes(taskUpdateRequestJson.length);
- }
- if (!dynamicFilterDomains.getDynamicFilterDomains().isEmpty()) {
- stats.updateWithDynamicFilterBytes(taskUpdateRequestJson.length);
- }
+ List splitAssignments = getSplitAssignments();
+ VersionedDynamicFilterDomains dynamicFilterDomains = outboundDynamicFiltersCollector.acknowledgeAndGetNewDomains(sentDynamicFiltersVersion.get());
- HttpUriBuilder uriBuilder = getHttpUriBuilder(taskStatus);
- Request request = preparePost()
- .setUri(uriBuilder.build())
- .setHeader(HttpHeaders.CONTENT_TYPE, MediaType.JSON_UTF_8.toString())
- .setBodyGenerator(createStaticBodyGenerator(taskUpdateRequestJson))
- .build();
+ // Workers don't need the embedded JSON representation when the fragment is sent
+ Optional fragment = sendPlan.get() ? Optional.of(planFragment.withoutEmbeddedJsonRepresentation()) : Optional.empty();
+ TaskUpdateRequest updateRequest = new TaskUpdateRequest(
+ session.toSessionRepresentation(),
+ session.getIdentity().getExtraCredentials(),
+ fragment,
+ splitAssignments,
+ outputBuffers.get(),
+ dynamicFilterDomains.getDynamicFilterDomains());
+ byte[] taskUpdateRequestJson = taskUpdateRequestCodec.toJsonBytes(updateRequest);
+ if (fragment.isPresent()) {
+ stats.updateWithPlanBytes(taskUpdateRequestJson.length);
+ }
+ if (!dynamicFilterDomains.getDynamicFilterDomains().isEmpty()) {
+ stats.updateWithDynamicFilterBytes(taskUpdateRequestJson.length);
+ }
- updateErrorTracker.startRequest();
+ HttpUriBuilder uriBuilder = getHttpUriBuilder(taskStatus);
+ Request request = preparePost()
+ .setUri(uriBuilder.build())
+ .setHeader(HttpHeaders.CONTENT_TYPE, MediaType.JSON_UTF_8.toString())
+ .setBodyGenerator(createStaticBodyGenerator(taskUpdateRequestJson))
+ .build();
- ListenableFuture> future = httpClient.executeAsync(request, createFullJsonResponseHandler(taskInfoCodec));
- currentRequest = future;
- currentRequestStartNanos = System.nanoTime();
+ updateErrorTracker.startRequest();
- // if pendingRequestsCounter is still non-zero (e.g. because triggerUpdate was called in the meantime)
- // then the request Future callback will send a new update via sendUpdate method call
- pendingRequestsCounter.addAndGet(-currentPendingRequestsCounter);
+ ListenableFuture> future = httpClient.executeAsync(request, createFullJsonResponseHandler(taskInfoCodec));
+ checkState(currentRequest.getAndSet(future) == null, "There should be no previous request running");
- Futures.addCallback(
- future,
- new SimpleHttpResponseHandler<>(new UpdateResponseHandler(splitAssignments, dynamicFilterDomains.getVersion()), request.getUri(), stats),
- executor);
- }
+ Futures.addCallback(
+ future,
+ new SimpleHttpResponseHandler<>(new UpdateResponseHandler(splitAssignments, dynamicFilterDomains.getVersion(), System.nanoTime(), currentPendingRequestsCounter), request.getUri(), stats),
+ executor);
}
private synchronized List getSplitAssignments()
@@ -710,12 +697,9 @@ private void cleanUpTask()
outboundDynamicFiltersCollector.acknowledge(Long.MAX_VALUE);
// cancel pending request
- synchronized (pendingRequestsCounter) {
- if (currentRequest != null) {
- currentRequest.cancel(true);
- currentRequest = null;
- currentRequestStartNanos = 0;
- }
+ Future> request = currentRequest.getAndSet(null);
+ if (request != null) {
+ request.cancel(true);
}
taskStatusFetcher.stop();
@@ -915,34 +899,33 @@ private class UpdateResponseHandler
{
private final List splitAssignments;
private final long currentRequestDynamicFiltersVersion;
+ private final long currentRequestStartNanos;
+ private final int currentPendingRequestsCounter;
- private UpdateResponseHandler(List splitAssignments, long currentRequestDynamicFiltersVersion)
+ private UpdateResponseHandler(List splitAssignments, long currentRequestDynamicFiltersVersion, long currentRequestStartNanos, int currentPendingRequestsCounter)
{
this.splitAssignments = ImmutableList.copyOf(requireNonNull(splitAssignments, "splitAssignments is null"));
this.currentRequestDynamicFiltersVersion = currentRequestDynamicFiltersVersion;
+ this.currentRequestStartNanos = currentRequestStartNanos;
+ this.currentPendingRequestsCounter = currentPendingRequestsCounter;
}
@Override
public void success(TaskInfo value)
{
try (SetThreadName ignored = new SetThreadName("UpdateResponseHandler-%s", taskId)) {
- try {
- sentDynamicFiltersVersion.set(currentRequestDynamicFiltersVersion);
- // Remove dynamic filters which were successfully sent to free up memory
- outboundDynamicFiltersCollector.acknowledge(currentRequestDynamicFiltersVersion);
- sendPlan.set(value.isNeedsPlan());
- long currentRequestStartNanos;
- synchronized (pendingRequestsCounter) {
- currentRequest = null;
- currentRequestStartNanos = HttpRemoteTask.this.currentRequestStartNanos;
- }
- updateStats(currentRequestStartNanos);
- processTaskUpdate(value, splitAssignments);
- updateErrorTracker.requestSucceeded();
- }
- finally {
- sendUpdate();
+ sentDynamicFiltersVersion.set(currentRequestDynamicFiltersVersion);
+ // Remove dynamic filters which were successfully sent to free up memory
+ outboundDynamicFiltersCollector.acknowledge(currentRequestDynamicFiltersVersion);
+ sendPlan.set(value.isNeedsPlan());
+ currentRequest.set(null);
+ updateStats();
+ updateErrorTracker.requestSucceeded();
+ if (pendingRequestsCounter.addAndGet(-currentPendingRequestsCounter) > 0) {
+ // schedule an update because triggerUpdate was called in the meantime
+ scheduleUpdate();
}
+ processTaskUpdate(value, splitAssignments);
}
}
@@ -951,21 +934,17 @@ public void failed(Throwable cause)
{
try (SetThreadName ignored = new SetThreadName("UpdateResponseHandler-%s", taskId)) {
try {
- long currentRequestStartNanos;
- synchronized (pendingRequestsCounter) {
- currentRequest = null;
- currentRequestStartNanos = HttpRemoteTask.this.currentRequestStartNanos;
- }
- updateStats(currentRequestStartNanos);
-
- // on failure assume we need to update again
- pendingRequestsCounter.incrementAndGet();
+ currentRequest.set(null);
+ updateStats();
// if task not already done, record error
TaskStatus taskStatus = getTaskStatus();
if (!taskStatus.getState().isDone()) {
updateErrorTracker.requestFailed(cause);
}
+
+ // on failure assume we need to update again
+ scheduleUpdate();
}
catch (Error e) {
fail(e);
@@ -974,9 +953,6 @@ public void failed(Throwable cause)
catch (RuntimeException e) {
fail(e);
}
- finally {
- sendUpdate();
- }
}
}
@@ -988,7 +964,7 @@ public void fatal(Throwable cause)
}
}
- private void updateStats(long currentRequestStartNanos)
+ private void updateStats()
{
Duration requestRoundTrip = Duration.nanosSince(currentRequestStartNanos);
stats.updateRoundTripMillis(requestRoundTrip.toMillis());
diff --git a/core/trino-main/src/main/java/io/trino/server/security/oauth2/JweTokenSerializer.java b/core/trino-main/src/main/java/io/trino/server/security/oauth2/JweTokenSerializer.java
index eda58cab016d..7c9b4f47a435 100644
--- a/core/trino-main/src/main/java/io/trino/server/security/oauth2/JweTokenSerializer.java
+++ b/core/trino-main/src/main/java/io/trino/server/security/oauth2/JweTokenSerializer.java
@@ -39,7 +39,6 @@
import java.time.Clock;
import java.util.Date;
import java.util.Map;
-import java.util.Optional;
import static com.google.common.base.Preconditions.checkState;
import static io.trino.server.security.jwt.JwtUtil.newJwtBuilder;
@@ -122,11 +121,7 @@ public String serialize(TokenPair tokenPair)
{
requireNonNull(tokenPair, "tokenPair is null");
- Optional