diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index c46ca116b997..bdb66117c70d 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -26,6 +26,7 @@ import io.airlift.slice.Slice; import io.trino.Session; import io.trino.connector.system.GlobalSystemConnector; +import io.trino.execution.QueryManager; import io.trino.metadata.LanguageFunctionManager.RunAsIdentityLoader; import io.trino.security.AccessControl; import io.trino.security.InjectedConnectorAccessControl; @@ -126,6 +127,7 @@ import java.util.Locale; import java.util.Map; import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -190,6 +192,7 @@ public final class MetadataManager private final TableFunctionRegistry tableFunctionRegistry; private final TypeManager typeManager; private final TypeCoercion typeCoercion; + private final QueryManager queryManager; private final ConcurrentMap catalogsByQueryId = new ConcurrentHashMap<>(); @@ -201,13 +204,15 @@ public MetadataManager( GlobalFunctionCatalog globalFunctionCatalog, LanguageFunctionManager languageFunctionManager, TableFunctionRegistry tableFunctionRegistry, - TypeManager typeManager) + TypeManager typeManager, + QueryManager queryManager) { this.accessControl = requireNonNull(accessControl, "accessControl is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); functions = requireNonNull(globalFunctionCatalog, "globalFunctionCatalog is null"); functionResolver = new BuiltinFunctionResolver(this, typeManager, globalFunctionCatalog); this.typeCoercion = new TypeCoercion(typeManager::getType); + this.queryManager = requireNonNull(queryManager, "queryManager is null"); this.systemSecurityMetadata = requireNonNull(systemSecurityMetadata, "systemSecurityMetadata is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); @@ -473,11 +478,33 @@ public TableMetadata getTableMetadata(Session session, TableHandle tableHandle) @Override public TableStatistics getTableStatistics(Session session, TableHandle tableHandle) { - CatalogHandle catalogHandle = tableHandle.catalogHandle(); - ConnectorMetadata metadata = getMetadata(session, catalogHandle); - TableStatistics tableStatistics = metadata.getTableStatistics(session.toConnectorSession(catalogHandle), tableHandle.connectorHandle()); - verifyNotNull(tableStatistics, "%s returned null tableStatistics for %s", metadata, tableHandle); - return tableStatistics; + try { + CatalogHandle catalogHandle = tableHandle.catalogHandle(); + ConnectorMetadata metadata = getMetadata(session, catalogHandle); + TableStatistics tableStatistics = metadata.getTableStatistics(session.toConnectorSession(catalogHandle), tableHandle.connectorHandle()); + verifyNotNull(tableStatistics, "%s returned null tableStatistics for %s", metadata, tableHandle); + return tableStatistics; + } + catch (RuntimeException e) { + if (isQueryDone(session)) { + // getting statistics for finished query may result in many different execeptions being thrown. + // As we do not care about the result anyway mask it by returning empty statistics. + return TableStatistics.empty(); + } + throw e; + } + } + + private boolean isQueryDone(Session session) + { + boolean done; + try { + done = queryManager.getQueryState(session.getQueryId()).isDone(); + } + catch (NoSuchElementException ex) { + done = true; + } + return done; } @Override diff --git a/core/trino-main/src/main/java/io/trino/testing/NotImplementedQueryManager.java b/core/trino-main/src/main/java/io/trino/testing/NotImplementedQueryManager.java new file mode 100644 index 000000000000..010bbc46082b --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/testing/NotImplementedQueryManager.java @@ -0,0 +1,150 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.testing; + +import com.google.common.util.concurrent.ListenableFuture; +import io.trino.Session; +import io.trino.execution.QueryExecution; +import io.trino.execution.QueryInfo; +import io.trino.execution.QueryManager; +import io.trino.execution.QueryState; +import io.trino.execution.StageId; +import io.trino.execution.StateMachine; +import io.trino.execution.TaskId; +import io.trino.server.BasicQueryInfo; +import io.trino.server.ResultQueryInfo; +import io.trino.server.protocol.Slug; +import io.trino.spi.QueryId; + +import java.util.List; +import java.util.NoSuchElementException; +import java.util.function.Consumer; + +public class NotImplementedQueryManager + implements QueryManager +{ + @Override + public List getQueries() + { + throw new RuntimeException("not implemented"); + } + + @Override + public void setOutputInfoListener(QueryId queryId, Consumer listener) + throws NoSuchElementException + { + throw new RuntimeException("not implemented"); + } + + @Override + public void outputTaskFailed(TaskId taskId, Throwable failure) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void resultsConsumed(QueryId queryId) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void addStateChangeListener(QueryId queryId, StateMachine.StateChangeListener listener) + throws NoSuchElementException + { + throw new RuntimeException("not implemented"); + } + + @Override + public ListenableFuture getStateChange(QueryId queryId, QueryState currentState) + { + throw new RuntimeException("not implemented"); + } + + @Override + public BasicQueryInfo getQueryInfo(QueryId queryId) + throws NoSuchElementException + { + throw new RuntimeException("not implemented"); + } + + @Override + public QueryInfo getFullQueryInfo(QueryId queryId) + throws NoSuchElementException + { + throw new RuntimeException("not implemented"); + } + + @Override + public ResultQueryInfo getResultQueryInfo(QueryId queryId) + throws NoSuchElementException + { + throw new RuntimeException("not implemented"); + } + + @Override + public Session getQuerySession(QueryId queryId) + { + throw new RuntimeException("not implemented"); + } + + @Override + public Slug getQuerySlug(QueryId queryId) + { + throw new RuntimeException("not implemented"); + } + + @Override + public QueryState getQueryState(QueryId queryId) + throws NoSuchElementException + { + throw new RuntimeException("not implemented"); + } + + @Override + public boolean hasQuery(QueryId queryId) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void recordHeartbeat(QueryId queryId) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void createQuery(QueryExecution execution) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void failQuery(QueryId queryId, Throwable cause) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void cancelQuery(QueryId queryId) + { + throw new RuntimeException("not implemented"); + } + + @Override + public void cancelStage(StageId stageId) + { + throw new RuntimeException("not implemented"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/testing/PlanTester.java b/core/trino-main/src/main/java/io/trino/testing/PlanTester.java index dfd338354622..af5122e74d71 100644 --- a/core/trino-main/src/main/java/io/trino/testing/PlanTester.java +++ b/core/trino-main/src/main/java/io/trino/testing/PlanTester.java @@ -366,7 +366,8 @@ private PlanTester(Session defaultSession, int nodeCountForStats) globalFunctionCatalog, languageFunctionManager, tableFunctionRegistry, - typeManager); + typeManager, + new NotImplementedQueryManager()); typeRegistry.addType(new JsonPath2016Type(new TypeDeserializer(typeManager), blockEncodingSerde)); this.joinCompiler = new JoinCompiler(typeOperators); this.hashStrategyCompiler = new FlatHashStrategyCompiler(typeOperators); diff --git a/core/trino-main/src/test/java/io/trino/metadata/TestMetadataManager.java b/core/trino-main/src/test/java/io/trino/metadata/TestMetadataManager.java index 8afab7c6d1f8..c1cc1f504fc0 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/TestMetadataManager.java +++ b/core/trino-main/src/test/java/io/trino/metadata/TestMetadataManager.java @@ -21,6 +21,7 @@ import io.trino.spi.type.TypeManager; import io.trino.spi.type.TypeOperators; import io.trino.sql.parser.SqlParser; +import io.trino.testing.NotImplementedQueryManager; import io.trino.transaction.TransactionManager; import io.trino.type.BlockTypeOperators; @@ -107,7 +108,8 @@ public MetadataManager build() globalFunctionCatalog, languageFunctionManager, tableFunctionRegistry, - typeManager); + typeManager, + new NotImplementedQueryManager()); } } }