-
Notifications
You must be signed in to change notification settings - Fork 3.4k
Add per catalog metastore metrics to QueryStats #26900
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
b08eb4d
82c9d70
af82bcf
9fd6e69
e2f4ff5
107045c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,13 +29,15 @@ | |
import io.opentelemetry.api.common.Attributes; | ||
import io.opentelemetry.api.trace.Span; | ||
import io.opentelemetry.api.trace.StatusCode; | ||
import io.trino.NotInTransactionException; | ||
import io.trino.Session; | ||
import io.trino.client.NodeVersion; | ||
import io.trino.exchange.ExchangeInput; | ||
import io.trino.execution.QueryExecution.QueryOutputInfo; | ||
import io.trino.execution.StateMachine.StateChangeListener; | ||
import io.trino.execution.querystats.PlanOptimizersStatsCollector; | ||
import io.trino.execution.warnings.WarningCollector; | ||
import io.trino.metadata.CatalogInfo; | ||
import io.trino.metadata.Metadata; | ||
import io.trino.operator.BlockedReason; | ||
import io.trino.operator.OperatorStats; | ||
|
@@ -182,6 +184,7 @@ public class QueryStateMachine | |
private final AtomicReference<Optional<Output>> output = new AtomicReference<>(Optional.empty()); | ||
private final AtomicReference<List<TableInfo>> referencedTables = new AtomicReference<>(ImmutableList.of()); | ||
private final AtomicReference<List<RoutineInfo>> routines = new AtomicReference<>(ImmutableList.of()); | ||
private final AtomicReference<Map<String, Metrics>> catalogMetadataMetrics = new AtomicReference<>(ImmutableMap.of()); | ||
private final StateMachine<Optional<QueryInfo>> finalQueryInfo; | ||
|
||
private final WarningCollector warningCollector; | ||
|
@@ -386,6 +389,38 @@ static QueryStateMachine beginWithTicker( | |
return queryStateMachine; | ||
} | ||
|
||
private void collectCatalogMetadataMetrics() | ||
{ | ||
try { | ||
if (session.getTransactionId().filter(transactionManager::transactionExists).isEmpty()) { | ||
// The metrics collection depends on active transaction as the metrics | ||
// are stored in the transactional ConnectorMetadata, but the collection can be | ||
// run after the query has failed e.g., via cancel. | ||
return; | ||
} | ||
|
||
ImmutableMap.Builder<String, Metrics> catalogMetadataMetrics = ImmutableMap.builder(); | ||
List<CatalogInfo> activeCatalogs = metadata.listActiveCatalogs(session); | ||
for (CatalogInfo activeCatalog : activeCatalogs) { | ||
Metrics metrics = metadata.getMetrics(session, activeCatalog.catalogName()); | ||
if (!metrics.getMetrics().isEmpty()) { | ||
catalogMetadataMetrics.put(activeCatalog.catalogName(), metrics); | ||
} | ||
} | ||
|
||
this.catalogMetadataMetrics.set(catalogMetadataMetrics.buildOrThrow()); | ||
} | ||
catch (NotInTransactionException e) { | ||
// Ignore, The metrics collection depends on an active transaction and should be skipped | ||
// if there is no one running. | ||
// The exception can be thrown even though there is a check for transaction at the top of the method, because | ||
// the transaction can be committed or aborted concurrently, after the check is done. | ||
} | ||
catch (RuntimeException e) { | ||
QUERY_STATE_LOG.error(e, "Error collecting query catalog metadata metrics: %s", queryId); | ||
} | ||
} | ||
|
||
public QueryId getQueryId() | ||
{ | ||
return queryId; | ||
|
@@ -851,6 +886,10 @@ private QueryStats getQueryStats(Optional<StagesInfo> stages) | |
} | ||
} | ||
|
||
// Try to collect the catalog metadata metrics. | ||
// The collection will fail, and we will use metrics collected earlier if | ||
// the query is already committed or aborted and metadata is not available. | ||
collectCatalogMetadataMetrics(); | ||
return new QueryStats( | ||
queryStateTimer.getCreateTime(), | ||
getExecutionStartTime().orElse(null), | ||
|
@@ -936,7 +975,7 @@ private QueryStats getQueryStats(Optional<StagesInfo> stages) | |
stageGcStatistics.build(), | ||
|
||
getDynamicFiltersStats(), | ||
|
||
catalogMetadataMetrics.get(), | ||
operatorStatsSummary.build(), | ||
planOptimizersStatsCollector.getTopRuleStats()); | ||
} | ||
|
@@ -1167,6 +1206,7 @@ public boolean transitionToFinishing() | |
transitionToFailed(e); | ||
return true; | ||
} | ||
collectCatalogMetadataMetrics(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We make metadata calls during FINISHING state as well, so we should collect metrics at the end of that There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We remove the transaction right after this call, so there is no way to collect the metrics anymore. |
||
|
||
Optional<TransactionInfo> transaction = session.getTransactionId().flatMap(transactionManager::getTransactionInfoIfExist); | ||
if (transaction.isPresent() && transaction.get().isAutoCommitContext()) { | ||
|
@@ -1244,6 +1284,7 @@ private boolean transitionToFailed(Throwable throwable, boolean log) | |
} | ||
return false; | ||
} | ||
collectCatalogMetadataMetrics(); | ||
|
||
try { | ||
if (log) { | ||
|
@@ -1549,6 +1590,7 @@ private static QueryStats pruneQueryStats(QueryStats queryStats) | |
queryStats.getFailedPhysicalWrittenDataSize(), | ||
queryStats.getStageGcStatistics(), | ||
queryStats.getDynamicFiltersStats(), | ||
ImmutableMap.of(), | ||
ImmutableList.of(), // Remove the operator summaries as OperatorInfo (especially DirectExchangeClientStatus) can hold onto a large amount of memory | ||
ImmutableList.of()); | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -67,6 +67,7 @@ | |
import io.trino.spi.function.FunctionMetadata; | ||
import io.trino.spi.function.LanguageFunction; | ||
import io.trino.spi.function.OperatorType; | ||
import io.trino.spi.metrics.Metrics; | ||
import io.trino.spi.predicate.TupleDomain; | ||
import io.trino.spi.security.FunctionAuthorization; | ||
import io.trino.spi.security.GrantInfo; | ||
|
@@ -147,6 +148,11 @@ Optional<TableExecuteHandle> getTableHandleForExecute( | |
|
||
Optional<Object> getInfo(Session session, TableHandle handle); | ||
|
||
/** | ||
* Return connector-specific, metadata operations metrics for the given session. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. connector -> catalog There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Well, the metrics are connector-specific, but we return them for the provided catalog. |
||
*/ | ||
Metrics getMetrics(Session session, String catalogName); | ||
|
||
CatalogSchemaTableName getTableName(Session session, TableHandle tableHandle); | ||
|
||
/** | ||
|
@@ -483,6 +489,8 @@ Optional<ConnectorOutputMetadata> finishRefreshMaterializedView( | |
*/ | ||
List<CatalogInfo> listCatalogs(Session session); | ||
|
||
List<CatalogInfo> listActiveCatalogs(Session session); | ||
|
||
/** | ||
* Get the names that match the specified table prefix (never null). | ||
*/ | ||
|
Uh oh!
There was an error while loading. Please reload this page.