if (!finalInfo.isPresent() || !finalInfo.get().getOutputStage().isPresent()) { return; Optional<StageInfo> prunedOutputStage = queryInfo.getOutputStage().map(outputStage -> new StageInfo( outputStage.getStageId(), outputStage.getState(), queryInfo.getQueryId(), queryInfo.getSession(), queryInfo.getState(), getMemoryPool().getId(), queryInfo.isScheduled(), queryInfo.getSelf(), queryInfo.getFieldNames(), queryInfo.getQuery(), pruneQueryStats(queryInfo.getQueryStats()), queryInfo.getSetCatalog(), queryInfo.getSetSchema(), queryInfo.getSetPath(), queryInfo.getSetSessionProperties(), queryInfo.getResetSessionProperties(), queryInfo.getSetRoles(), queryInfo.getAddedPreparedStatements(), queryInfo.getDeallocatedPreparedStatements(), queryInfo.getStartedTransactionId(), queryInfo.isClearTransactionId(), queryInfo.getUpdateType(), prunedOutputStage, queryInfo.getFailureInfo(),
public BasicQueryInfo(QueryInfo queryInfo) { this(queryInfo.getQueryId(), queryInfo.getSession(), queryInfo.getResourceGroupId(), queryInfo.getState(), queryInfo.getMemoryPool(), queryInfo.isScheduled(), queryInfo.getSelf(), queryInfo.getQuery(), new BasicQueryStats(queryInfo.getQueryStats()), queryInfo.getErrorType(), queryInfo.getErrorCode()); }
private synchronized void closeExchangeClientIfNecessary(QueryInfo queryInfo) { // Close the exchange client if the query has failed, or if the query // is done and it does not have an output stage. The latter happens // for data definition executions, as those do not have output. if ((queryInfo.getState() == FAILED) || (queryInfo.getState().isDone() && !queryInfo.getOutputStage().isPresent())) { exchangeClient.close(); } }
private QueryMetadata createQueryMetadata(QueryInfo queryInfo) { return new QueryMetadata( queryInfo.getQueryId().toString(), queryInfo.getSession().getTransactionId().map(TransactionId::toString), queryInfo.getQuery(), queryInfo.getState().toString(), queryInfo.getSelf(), createTextQueryPlan(queryInfo), queryInfo.getOutputStage().flatMap(stage -> stageInfoCodec.toJsonWithLengthLimit(stage, maxJsonLimit))); }
assertEquals(queryInfo.getQueryId(), TEST_SESSION.getQueryId()); assertEquals(queryInfo.getSelf(), LOCATION); assertFalse(queryInfo.getOutputStage().isPresent()); assertEquals(queryInfo.getQuery(), QUERY); assertEquals(queryInfo.getInputs(), INPUTS); assertEquals(queryInfo.getOutput(), OUTPUT); assertEquals(queryInfo.getFieldNames(), OUTPUT_FIELD_NAMES); assertEquals(queryInfo.getUpdateType(), UPDATE_TYPE); assertEquals(queryInfo.getMemoryPool(), MEMORY_POOL.getId()); QueryStats queryStats = queryInfo.getQueryStats(); assertNotNull(queryStats.getElapsedTime()); assertNotNull(queryStats.getQueuedTime()); if (queryInfo.getState() == QUEUED || queryInfo.getState() == WAITING_FOR_RESOURCES) { assertNull(queryStats.getExecutionStartTime()); if (queryInfo.getState().isDone()) { assertNotNull(queryStats.getEndTime()); assertEquals(queryInfo.getState(), expectedState); assertEquals(stateMachine.isDone(), expectedState.isDone()); assertNotNull(queryInfo.getFailureInfo()); FailureInfo failure = queryInfo.getFailureInfo().toFailureInfo(); assertNotNull(failure); assertEquals(failure.getType(), expectedException.getClass().getName()); if (expectedException instanceof PrestoException) { assertEquals(queryInfo.getErrorCode(), ((PrestoException) expectedException).getErrorCode());
session = queryManager.getFullQueryInfo(queryId).getSession().toSession(sessionPropertyManager); serde = new PagesSerdeFactory(blockEncodingSerde, isExchangeCompressionEnabled(session)).createPagesSerde(); if ((data != null) && (queryInfo.getUpdateType() != null) && (updateCount == null) && (columns.size() == 1) && (columns.get(0).getType().equals(StandardTypes.BIGINT))) { Iterator<List<Object>> iterator = data.iterator(); if ((queryInfo.getState() == QueryState.FINISHED) && !queryInfo.getOutputStage().isPresent()) { columns = ImmutableList.of(createColumn("result", BooleanType.BOOLEAN)); data = ImmutableSet.of(ImmutableList.of(true)); if (!queryInfo.isFinalQueryInfo() && !queryInfo.getState().equals(QueryState.FAILED) || !exchangeClient.isClosed()) { nextResultsUri = createNextResultsUri(scheme, uriInfo); setCatalog = queryInfo.getSetCatalog(); setSchema = queryInfo.getSetSchema(); setPath = queryInfo.getSetPath(); setSessionProperties = queryInfo.getSetSessionProperties(); resetSessionProperties = queryInfo.getResetSessionProperties(); setRoles = queryInfo.getSetRoles(); addedPreparedStatements = queryInfo.getAddedPreparedStatements(); deallocatedPreparedStatements = queryInfo.getDeallocatedPreparedStatements(); startedTransactionId = queryInfo.getStartedTransactionId(); clearTransactionId = queryInfo.isClearTransactionId();
@Override public DateTime getLastHeartbeat() { return queryInfo.getQueryStats().getLastHeartbeat(); }
.collect(toImmutableList()); for (QueryInfo queryInfo : queryInfos) { QueryStats queryStats = queryInfo.getQueryStats(); table.addRow( queryInfo.getQueryId().toString(), queryInfo.getState().toString(), queryInfo.getSession().getUser(), queryInfo.getSession().getSource().orElse(null), queryInfo.getQuery(), queryInfo.getResourceGroupId().map(QuerySystemTable::resourceGroupIdToBlock).orElse(null),
public void queryCompletedEvent(QueryInfo queryInfo) { QueryStats queryStats = queryInfo.getQueryStats(); eventListenerManager.queryCompleted( new QueryCompletedEvent( createQueryMetadata(queryInfo), createQueryStatistics(queryInfo), createQueryContext(queryInfo.getSession(), queryInfo.getResourceGroupId()), getQueryIOMetadata(queryInfo), createQueryFailureInfo(queryInfo.getFailureInfo(), queryInfo.getOutputStage()), queryInfo.getWarnings(), ofEpochMilli(queryStats.getCreateTime().getMillis()), ofEpochMilli(queryStats.getExecutionStartTime().getMillis()), ofEpochMilli(queryStats.getEndTime() != null ? queryStats.getEndTime().getMillis() : 0))); logQueryTimeline(queryInfo); }
private static QueryError toQueryError(QueryInfo queryInfo) QueryState state = queryInfo.getState(); if (state != FAILED) { return null; if (queryInfo.getFailureInfo() != null) { failure = queryInfo.getFailureInfo().toFailureInfo(); log.warn("Query %s in state %s has no failure info", queryInfo.getQueryId(), state); failure = toFailure(new RuntimeException(format("Query is %s (reason unknown)", state))).toFailureInfo(); if (queryInfo.getErrorCode() != null) { errorCode = queryInfo.getErrorCode(); log.warn("Failed query %s has no error code", queryInfo.getQueryId());
private static StatementStats toStatementStats(QueryInfo queryInfo) { QueryStats queryStats = queryInfo.getQueryStats(); StageInfo outputStage = queryInfo.getOutputStage().orElse(null); return StatementStats.builder() .setState(queryInfo.getState().toString()) .setQueued(queryInfo.getState() == QueryState.QUEUED) .setScheduled(queryInfo.isScheduled()) .setNodes(globalUniqueNodes(outputStage).size()) .setTotalSplits(queryStats.getTotalDrivers()) .setQueuedSplits(queryStats.getQueuedDrivers()) .setRunningSplits(queryStats.getRunningDrivers() + queryStats.getBlockedDrivers()) .setCompletedSplits(queryStats.getCompletedDrivers()) .setCpuTimeMillis(queryStats.getTotalCpuTime().toMillis()) .setWallTimeMillis(queryStats.getTotalScheduledTime().toMillis()) .setQueuedTimeMillis(queryStats.getQueuedTime().toMillis()) .setElapsedTimeMillis(queryStats.getElapsedTime().toMillis()) .setProcessedRows(queryStats.getRawInputPositions()) .setProcessedBytes(queryStats.getRawInputDataSize().toBytes()) .setPeakMemoryBytes(queryStats.getPeakUserMemoryReservation().toBytes()) .setRootStage(toStageStats(outputStage)) .build(); }
private static void logQueryTimeline(QueryInfo queryInfo) QueryStats queryStats = queryInfo.getQueryStats(); DateTime queryStartTime = queryStats.getCreateTime(); DateTime queryEndTime = queryStats.getEndTime(); List<StageInfo> stages = StageInfo.getAllStages(queryInfo.getOutputStage()); queryInfo.getQueryId(), queryInfo.getSession().getTransactionId().map(TransactionId::toString).orElse(""), elapsed, planning,
@Test(timeOut = 60_000L) public void testFailQuery() throws Exception { QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); QueryId queryId = queryManager.createQueryId(); queryManager.createQuery( queryId, new TestingSessionContext(TEST_SESSION), "SELECT * FROM lineitem") .get(); // wait until query starts running while (true) { QueryState state = queryManager.getQueryState(queryId); if (state.isDone()) { fail("unexpected query state: " + state); } if (state == RUNNING) { break; } Thread.sleep(100); } // cancel query queryManager.failQuery(queryId, new PrestoException(GENERIC_INTERNAL_ERROR, "mock exception")); QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId); assertEquals(queryInfo.getState(), FAILED); assertEquals(queryInfo.getErrorCode(), GENERIC_INTERNAL_ERROR.toErrorCode()); assertNotNull(queryInfo.getFailureInfo()); assertEquals(queryInfo.getFailureInfo().getMessage(), "mock exception"); }
private static QueryIOMetadata getQueryIOMetadata(QueryInfo queryInfo) { ImmutableList.Builder<QueryInputMetadata> inputs = ImmutableList.builder(); for (Input input : queryInfo.getInputs()) { inputs.add(new QueryInputMetadata( input.getConnectorId().getCatalogName(), input.getSchema(), input.getTable(), input.getColumns().stream() .map(Column::getName).collect(Collectors.toList()), input.getConnectorInfo())); } Optional<QueryOutputMetadata> output = Optional.empty(); if (queryInfo.getOutput().isPresent()) { Optional<TableFinishInfo> tableFinishInfo = queryInfo.getQueryStats().getOperatorSummaries().stream() .map(OperatorStats::getInfo) .filter(TableFinishInfo.class::isInstance) .map(TableFinishInfo.class::cast) .findFirst(); output = Optional.of( new QueryOutputMetadata( queryInfo.getOutput().get().getConnectorId().getCatalogName(), queryInfo.getOutput().get().getSchema(), queryInfo.getOutput().get().getTable(), tableFinishInfo.map(TableFinishInfo::getConnectorOutputMetadata), tableFinishInfo.map(TableFinishInfo::isJsonLengthLimitExceeded))); } return new QueryIOMetadata(inputs.build(), output); }
long rawInputBytes = info.getQueryStats().getRawInputDataSize().toBytes(); consumedCpuTimeSecs.update((long) info.getQueryStats().getTotalCpuTime().getValue(SECONDS)); consumedInputBytes.update(info.getQueryStats().getRawInputDataSize().toBytes()); consumedInputRows.update(info.getQueryStats().getRawInputPositions()); executionTime.add(info.getQueryStats().getExecutionTime()); queuedTime.add(info.getQueryStats().getQueuedTime()); long executionWallMillis = info.getQueryStats().getExecutionTime().toMillis(); if (executionWallMillis > 0) { wallInputBytesRate.add(rawInputBytes * 1000 / executionWallMillis); long executionCpuMillis = info.getQueryStats().getTotalCpuTime().toMillis(); if (executionCpuMillis > 0) { cpuInputByteRate.add(rawInputBytes * 1000 / executionCpuMillis); if (info.getErrorCode() != null) { switch (info.getErrorCode().getType()) { case USER_ERROR: userErrorFailures.update(1); if (info.getErrorCode().getCode() == ABANDONED_QUERY.toErrorCode().getCode()) { abandonedQueries.update(1); else if (info.getErrorCode().getCode() == USER_CANCELED.toErrorCode().getCode()) { canceledQueries.update(1);
private QueryStatistics createQueryStatistics(QueryInfo queryInfo) for (OperatorStats summary : queryInfo.getQueryStats().getOperatorSummaries()) { operatorSummaries.add(operatorStatsCodec.toJson(summary)); Optional<StatsAndCosts> planNodeStatsAndCosts = queryInfo.getOutputStage().map(QueryMonitor::reconstructStatsAndCosts); Optional<String> serializedPlanNodeStatsAndCosts = planNodeStatsAndCosts.map(statsAndCostsCodec::toJson); QueryStats queryStats = queryInfo.getQueryStats(); return new QueryStatistics( ofMillis(queryStats.getTotalCpuTime().toMillis()), queryStats.getStageGcStatistics(), queryStats.getCompletedDrivers(), queryInfo.isCompleteInfo(), getCpuDistributions(queryInfo), operatorSummaries.build(),
@Test public void testMetadataIsClearedAfterQueryCanceled() throws Exception { QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); QueryId queryId = queryManager.createQueryId(); queryManager.createQuery( queryId, new TestingSessionContext(TEST_SESSION), "SELECT * FROM lineitem") .get(); // wait until query starts running while (true) { QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId); if (queryInfo.getState().isDone()) { assertEquals(queryInfo.getState(), FAILED); throw queryInfo.getFailureInfo().toException(); } if (queryInfo.getState() == RUNNING) { break; } Thread.sleep(100); } // cancel query queryManager.cancelQuery(queryId); assertEquals(metadataManager.getCatalogsByQueryId().size(), 0); }
private Optional<String> createTextQueryPlan(QueryInfo queryInfo) { try { if (queryInfo.getOutputStage().isPresent()) { return Optional.of(textDistributedPlan( queryInfo.getOutputStage().get(), functionRegistry, queryInfo.getSession().toSession(sessionPropertyManager), false)); } } catch (Exception e) { // Sometimes it is expected to fail. For example if generated plan is too long. // Don't fail to create event if the plan can not be created. log.warn(e, "Error creating explain plan for query %s", queryInfo.getQueryId()); } return Optional.empty(); }
private QueryStatistics createQueryStatistics(QueryInfo queryInfo) { ImmutableList.Builder<String> operatorSummaries = ImmutableList.builder(); for (OperatorStats summary : queryInfo.getQueryStats().getOperatorSummaries()) { operatorSummaries.add(operatorStatsCodec.toJson(summary)); } QueryStats queryStats = queryInfo.getQueryStats(); return new QueryStatistics( ofMillis(queryStats.getTotalCpuTime().toMillis()), ofMillis(queryStats.getTotalScheduledTime().toMillis()), ofMillis(queryStats.getQueuedTime().toMillis()), Optional.of(ofMillis(queryStats.getAnalysisTime().toMillis())), Optional.of(ofMillis(queryStats.getDistributedPlanningTime().toMillis())), queryStats.getPeakUserMemoryReservation().toBytes(), queryStats.getPeakTotalMemoryReservation().toBytes(), queryStats.getPeakTaskTotalMemory().toBytes(), queryStats.getRawInputDataSize().toBytes(), queryStats.getRawInputPositions(), queryStats.getOutputDataSize().toBytes(), queryStats.getOutputPositions(), queryStats.getLogicalWrittenDataSize().toBytes(), queryStats.getWrittenPositions(), queryStats.getCumulativeUserMemory(), queryStats.getStageGcStatistics(), queryStats.getCompletedDrivers(), queryInfo.isCompleteInfo(), getCpuDistributions(queryInfo), operatorSummaries.build()); }
@Override public QueryState getState() { return queryInfo.getState(); }