@Override public List<BasicQueryInfo> getQueries() { return queryTracker.getAllQueries().stream() .map(queryExecution -> { try { return queryExecution.getBasicQueryInfo(); } catch (RuntimeException ignored) { return null; } }) .filter(Objects::nonNull) .collect(toImmutableList()); }
@Override public QueryState getQueryState(QueryId queryId) { return queryTracker.getQuery(queryId).getState(); }
public synchronized void start() { checkState(backgroundTask == null, "QueryTracker already started"); backgroundTask = queryManagementExecutor.scheduleWithFixedDelay(() -> { try { failAbandonedQueries(); } catch (Throwable e) { log.error(e, "Error cancelling abandoned queries"); } try { enforceTimeLimits(); } catch (Throwable e) { log.error(e, "Error enforcing query timeout limits"); } try { removeExpiredQueries(); } catch (Throwable e) { log.error(e, "Error removing expired queries"); } try { pruneExpiredQueries(); } catch (Throwable e) { log.error(e, "Error pruning expired queries"); } }, 1, 1, TimeUnit.SECONDS); }
requireNonNull(query, "query is null"); checkArgument(!query.isEmpty(), "query must not be empty string"); checkArgument(!queryTracker.tryGetQuery(queryId).isPresent(), "query %s already exists", queryId); queryTracker.addQuery(execution); queryTracker.expireQuery(queryId); queryTracker.expireQuery(queryId); if (!queryTracker.addQuery(queryExecution)) {
queryManagementExecutorMBean = new ThreadPoolExecutorMBean((ThreadPoolExecutor) queryManagementExecutor); this.queryTracker = new QueryTracker<>(queryManagerConfig, queryManagementExecutor);
private void failAbandonedQueries() { for (T query : queries.values()) { try { if (query.isDone()) { continue; } if (isAbandoned(query)) { log.info("Failing abandoned query %s", query.getQueryId()); query.fail(new PrestoException( ABANDONED_QUERY, format("Query %s has not been accessed since %s: currentTime %s", query.getQueryId(), query.getLastHeartbeat(), DateTime.now()))); } } catch (RuntimeException e) { log.error(e, "Exception failing abandoned query %s", query.getQueryId()); } } }
@Override public void addOutputInfoListener(QueryId queryId, Consumer<QueryOutputInfo> listener) { requireNonNull(listener, "listener is null"); queryTracker.getQuery(queryId).addOutputInfoListener(listener); }
/** * Enforce memory limits at the query level */ private void enforceMemoryLimits() { List<QueryExecution> runningQueries = queryTracker.getAllQueries().stream() .filter(query -> query.getState() == RUNNING) .collect(toImmutableList()); memoryManager.process(runningQueries, this::getQueries); }
@Override public QueryInfo getFullQueryInfo(QueryId queryId) { return queryTracker.getQuery(queryId).getQueryInfo(); }
/** * Enforce query CPU time limits */ private void enforceCpuLimits() { for (QueryExecution query : queryTracker.getAllQueries()) { Duration cpuTime = query.getTotalCpuTime(); Duration sessionLimit = getQueryMaxCpuTime(query.getSession()); Duration limit = Ordering.natural().min(maxQueryCpuTime, sessionLimit); if (cpuTime.compareTo(limit) > 0) { query.fail(new ExceededCpuLimitException(limit)); } } }
@Override public void addStateChangeListener(QueryId queryId, StateChangeListener<QueryState> listener) { requireNonNull(listener, "listener is null"); queryTracker.getQuery(queryId).addStateChangeListener(listener); }
@Override public BasicQueryInfo getQueryInfo(QueryId queryId) { return queryTracker.getQuery(queryId).getBasicQueryInfo(); }
public Plan getQueryPlan(QueryId queryId) { return queryTracker.getQuery(queryId).getQueryPlan(); }
public void addFinalQueryInfoListener(QueryId queryId, StateChangeListener<QueryInfo> stateChangeListener) { queryTracker.getQuery(queryId).addFinalQueryInfoListener(stateChangeListener); }