private void testRejection() throws Exception { try (DistributedQueryRunner queryRunner = createQueryRunner()) { queryRunner.installPlugin(new ResourceGroupManagerPlugin()); queryRunner.getCoordinator().getResourceGroupManager().get().setConfigurationManager("file", ImmutableMap.of("resource-groups.config-file", getResourceFilePath("resource_groups_config_dashboard.json"))); QueryId queryId = createQuery(queryRunner, newRejectionSession(), LONG_LASTING_QUERY); waitForQueryState(queryRunner, queryId, FAILED); QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); assertEquals(queryManager.getQueryInfo(queryId).getErrorCode(), QUERY_REJECTED.toErrorCode()); } }
@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 void testNoLeak(@Language("SQL") String query) throws Exception { Map<String, String> properties = ImmutableMap.<String, String>builder() .put("task.verbose-stats", "true") .build(); try (DistributedQueryRunner queryRunner = createQueryRunner(TINY_SESSION, properties)) { executor.submit(() -> queryRunner.execute(query)).get(); for (BasicQueryInfo info : queryRunner.getCoordinator().getQueryManager().getQueries()) { assertEquals(info.getState(), FINISHED); } // Make sure we didn't leak any memory on the workers for (TestingPrestoServer worker : queryRunner.getServers()) { Optional<MemoryPool> reserved = worker.getLocalMemoryManager().getReservedPool(); assertTrue(reserved.isPresent()); assertEquals(reserved.get().getMaxBytes(), reserved.get().getFreeBytes()); MemoryPool general = worker.getLocalMemoryManager().getGeneralPool(); assertEquals(general.getMaxBytes(), general.getFreeBytes()); } } }
private void cancelAllQueries() { QueryManager queryManager = coordinator.getQueryManager(); for (BasicQueryInfo queryInfo : queryManager.getQueries()) { if (!queryInfo.getState().isDone()) { queryManager.cancelQuery(queryInfo.getQueryId()); } } }
private Response failQuery(QueryId queryId, PrestoException queryException) { requireNonNull(queryId, "queryId is null"); try { QueryState state = queryManager.getQueryState(queryId); // check before killing to provide the proper error code (this is racy) if (state.isDone()) { return Response.status(Status.CONFLICT).build(); } queryManager.failQuery(queryId, queryException); // verify if the query was failed (if not, we lost the race) if (!queryException.getErrorCode().equals(queryManager.getQueryInfo(queryId).getErrorCode())) { return Response.status(Status.CONFLICT).build(); } return Response.status(Status.OK).build(); } catch (NoSuchElementException e) { return Response.status(Status.GONE).build(); } }
private synchronized ListenableFuture<?> getFutureStateChange() { // ensure the query has been submitted submissionFuture.submitQuery(); // if query query submission has not finished, wait for it to finish if (!submissionFuture.isDone()) { return submissionFuture; } // if the exchange client is open, wait for data if (!exchangeClient.isClosed()) { return exchangeClient.isBlocked(); } // otherwise, wait for the query to finish queryManager.recordHeartbeat(queryId); try { return queryDoneFuture(queryManager.getQueryState(queryId)); } catch (NoSuchElementException e) { return immediateFuture(null); } }
private void assertResourceGroup(DistributedQueryRunner queryRunner, Session session, String query, ResourceGroupId expectedResourceGroup) throws InterruptedException { QueryId queryId = createQuery(queryRunner, session, query); waitForQueryState(queryRunner, queryId, ImmutableSet.of(RUNNING, FINISHED)); Optional<ResourceGroupId> resourceGroupId = queryRunner.getCoordinator().getQueryManager().getFullQueryInfo(queryId).getResourceGroupId(); assertTrue(resourceGroupId.isPresent(), "Query should have a resource group"); assertEquals(resourceGroupId.get(), expectedResourceGroup, format("Expected: '%s' resource group, found: %s", expectedResourceGroup, resourceGroupId.get())); }
throws Exception Map<String, String> properties = ImmutableMap.<String, String>builder() .put("task.verbose-stats", "true") .build(); queryFutures.add(executor.submit(() -> queryRunner.execute("SELECT COUNT(*), clerk FROM orders GROUP BY clerk"))); assertNotNull(generalPool); List<BasicQueryInfo> currentQueryInfos = queryRunner.getCoordinator().getQueryManager().getQueries(); for (BasicQueryInfo info : currentQueryInfos) { assertFalse(info.getState().isDone()); assertEquals(currentQueryInfos.size(), 2); currentQueryInfos = queryRunner.getCoordinator().getQueryManager().getQueries(); for (BasicQueryInfo info : currentQueryInfos) { assertFalse(info.getState().isDone()); reserved.get().free(fakeQueryId, "test", reserved.get().getMaxBytes()); assertTrue(reserved.get().getFreeBytes() > 0); for (BasicQueryInfo info : queryRunner.getCoordinator().getQueryManager().getQueries()) { assertEquals(info.getState(), FINISHED);
throws Exception Map<String, String> properties = ImmutableMap.<String, String>builder() .put("node-scheduler.include-coordinator", "false") .put("shutdown.grace-period", "10s") List<ListenableFuture<?>> queryFutures = new ArrayList<>(); for (int i = 0; i < 5; i++) { queryFutures.add(executor.submit(() -> queryRunner.execute("SELECT COUNT(*), clerk FROM orders GROUP BY clerk"))); TestingPrestoServer worker = queryRunner.getServers() .stream() .filter(server -> !server.isCoordinator()) .findFirst() .get(); TaskManager taskManager = worker.getTaskManager(); worker.getGracefulShutdownHandler().requestShutdown(); List<BasicQueryInfo> queryInfos = queryRunner.getCoordinator().getQueryManager().getQueries(); for (BasicQueryInfo info : queryInfos) { assertEquals(info.getState(), FINISHED); TestShutdownAction shutdownAction = (TestShutdownAction) worker.getShutdownAction(); shutdownAction.waitForShutdownComplete(SHUTDOWN_TIMEOUT_MILLIS); assertTrue(shutdownAction.isWorkerShutdown());
if (cachedResult.isPresent()) { return cachedResult.get(); session = queryManager.getFullQueryInfo(queryId).getSession().toSession(sessionPropertyManager); serde = new PagesSerdeFactory(blockEncodingSerde, isExchangeCompressionEnabled(session)).createPagesSerde(); queryManager.failQuery(queryId, cause); QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId); queryManager.recordHeartbeat(queryId); 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(new Column("result", BooleanType.BOOLEAN)); data = ImmutableSet.of(ImmutableList.of(true));
private synchronized Optional<QueryResults> getCachedResult(long token, UriInfo uriInfo) { // is the a repeated request for the last results? String requestedPath = uriInfo.getAbsolutePath().getPath(); if (requestedPath.equals(lastResultPath)) { if (submissionFuture.isDone()) { // tell query manager we are still interested in the query queryManager.recordHeartbeat(queryId); } return Optional.of(lastResult); } if (token < resultId.get()) { throw new WebApplicationException(Response.Status.GONE); } // if this is not a request for the next results, return not found if (lastResult.getNextUri() == null || !requestedPath.equals(lastResult.getNextUri().getPath())) { // unknown token throw new WebApplicationException(Response.Status.NOT_FOUND); } return Optional.empty(); }
@Override public void run() { try { // Queries are added to the query manager before being recorded in queryIds set. // Therefore, we take a snapshot if queryIds before getting the live queries // from the query manager. Then we remove only the queries in the snapshot and // not live queries set. If we did this in the other order, a query could be // registered between fetching the live queries and inspecting the queryIds set. for (QueryId queryId : ImmutableSet.copyOf(queries.keySet())) { Query query = queries.get(queryId); Optional<QueryState> state = queryManager.getQueryState(queryId); // free up resources if the query completed if (!state.isPresent() || state.get() == QueryState.FAILED) { query.dispose(); } // forget about this query if the query manager is no longer tracking it if (!state.isPresent()) { queries.remove(queryId); } } } catch (Throwable e) { log.warn(e, "Error removing old queries"); } } }
List<QueryInfo> queryInfos = queryManager.getQueries().stream() .map(BasicQueryInfo::getQueryId) .map(queryId -> { try { return queryManager.getFullQueryInfo(queryId); .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),
assertNull(queryResults.getError()); BasicQueryInfo queryInfo = server.getQueryManager().getQueryInfo(new QueryId(queryResults.getId())); assertEquals(queryInfo.getSession().getSystemProperties(), ImmutableMap.builder() .put(QUERY_MAX_MEMORY, "1GB") .put(JOIN_DISTRIBUTION_TYPE, "partitioned") assertEquals(queryInfo.getSession().getClientInfo().get(), "{\"clientVersion\":\"testVersion\"}"); assertEquals(queryInfo.getSession().getPreparedStatements(), ImmutableMap.builder() .put("foo", "select * from bar") .build());
QueryInfo queryInfo = server.getQueryManager().getQueryInfo(new QueryId(queryResults.getId())); assertEquals(queryInfo.getSession().getSystemProperties(), ImmutableMap.builder() .put(QUERY_MAX_MEMORY, "1GB") .put(DISTRIBUTED_JOIN, "true") assertEquals(rows, ImmutableList.of(ImmutableList.of("system")));
@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 void waitForQueryToBeKilled(DistributedQueryRunner queryRunner) throws InterruptedException { while (true) { for (BasicQueryInfo info : queryRunner.getCoordinator().getQueryManager().getQueries()) { if (info.getState().isDone()) { assertNotNull(info.getErrorCode()); assertEquals(info.getErrorCode().getCode(), CLUSTER_OUT_OF_MEMORY.toErrorCode().getCode()); return; } } MILLISECONDS.sleep(10); } }
query = manager.getQueryInfo(QueryId.valueOf(queryId)); List<StageInfo> stages = collectStages(query.getOutputStage()); Map<String, Object> result = ImmutableMap.<String, Object>builder() .put("tasks", tasks) .put("flows", flows)
@Test(timeOut = 60_000L) public void testQueryCpuLimit() throws Exception { try (DistributedQueryRunner queryRunner = builder().setSingleExtraProperty("query.max-cpu-time", "1ms").build()) { QueryId queryId = createQuery(queryRunner, TEST_SESSION, "SELECT COUNT(*) FROM lineitem"); waitForQueryState(queryRunner, queryId, FAILED); QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); BasicQueryInfo queryInfo = queryManager.getQueryInfo(queryId); assertEquals(queryInfo.getState(), FAILED); assertEquals(queryInfo.getErrorCode(), EXCEEDED_CPU_LIMIT.toErrorCode()); } } }
@Test(timeOut = 60_000L) public void testLegacyQueryContext() throws Exception { QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); QueryId queryId = queryManager.createQueryId(); queryManager.createQuery( queryId, new TestingSessionContext(TEST_SESSION), "SELECT * FROM lineitem") .get(); waitForQueryState(queryRunner, queryId, RUNNING); // cancel query queryManager.failQuery(queryId, new PrestoException(GENERIC_INTERNAL_ERROR, "mock exception")); // assert that LegacyQueryContext is used instead of the DefaultQueryContext SqlTaskManager taskManager = (SqlTaskManager) queryRunner.getServers().get(0).getTaskManager(); assertInstanceOf(taskManager.getQueryContext(queryId), LegacyQueryContext.class); } }