Skip to content

Commit

Permalink
Set up only one concurrent query runner in TestQueryManager
Browse files Browse the repository at this point in the history
`TestQueryManager` defined a default `queryRunner` used by one test
method, but other test methods created additional runners. This commit
moves the default one into the only test method that used it. In the
result, there is at most 1 query runner at the same time created during
test execution.
  • Loading branch information
findepi authored and wendigo committed Dec 21, 2023
1 parent 259702f commit 35ad5c1
Showing 1 changed file with 29 additions and 48 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,7 @@
import io.trino.testing.DistributedQueryRunner;
import io.trino.testing.TestingSessionContext;
import io.trino.tests.tpch.TpchQueryRunnerBuilder;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.parallel.Execution;

Expand All @@ -48,64 +45,48 @@
import static java.util.Arrays.stream;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Fail.fail;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD;

@TestInstance(PER_CLASS)
@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once
public class TestQueryManager
{
private DistributedQueryRunner queryRunner;

@BeforeAll
public void setUp()
throws Exception
{
queryRunner = TpchQueryRunnerBuilder.builder().build();
}

@AfterAll
public void tearDown()
{
queryRunner.close();
queryRunner = null;
}

@Test
@Timeout(60)
public void testFailQuery()
throws Exception
{
DispatchManager dispatchManager = queryRunner.getCoordinator().getDispatchManager();
QueryId queryId = dispatchManager.createQueryId();
dispatchManager.createQuery(
queryId,
Span.getInvalid(),
Slug.createNew(),
TestingSessionContext.fromSession(TEST_SESSION),
"SELECT * FROM lineitem")
.get();
try (DistributedQueryRunner queryRunner = TpchQueryRunnerBuilder.builder().build()) {
DispatchManager dispatchManager = queryRunner.getCoordinator().getDispatchManager();
QueryId queryId = dispatchManager.createQueryId();
dispatchManager.createQuery(
queryId,
Span.getInvalid(),
Slug.createNew(),
TestingSessionContext.fromSession(TEST_SESSION),
"SELECT * FROM lineitem")
.get();

// wait until query starts running
while (true) {
QueryState state = dispatchManager.getQueryInfo(queryId).getState();
if (state.isDone()) {
fail("unexpected query state: " + state);
// wait until query starts running
while (true) {
QueryState state = dispatchManager.getQueryInfo(queryId).getState();
if (state.isDone()) {
fail("unexpected query state: " + state);
}
if (state == RUNNING) {
break;
}
Thread.sleep(100);
}
if (state == RUNNING) {
break;
}
Thread.sleep(100);
}

// cancel query
QueryManager queryManager = queryRunner.getCoordinator().getQueryManager();
queryManager.failQuery(queryId, new TrinoException(GENERIC_INTERNAL_ERROR, "mock exception"));
QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId);
assertThat(queryInfo.getState()).isEqualTo(FAILED);
assertThat(queryInfo.getErrorCode()).isEqualTo(GENERIC_INTERNAL_ERROR.toErrorCode());
assertThat(queryInfo.getFailureInfo()).isNotNull();
assertThat(queryInfo.getFailureInfo().getMessage()).isEqualTo("mock exception");
// cancel query
QueryManager queryManager = queryRunner.getCoordinator().getQueryManager();
queryManager.failQuery(queryId, new TrinoException(GENERIC_INTERNAL_ERROR, "mock exception"));
QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId);
assertThat(queryInfo.getState()).isEqualTo(FAILED);
assertThat(queryInfo.getErrorCode()).isEqualTo(GENERIC_INTERNAL_ERROR.toErrorCode());
assertThat(queryInfo.getFailureInfo()).isNotNull();
assertThat(queryInfo.getFailureInfo().getMessage()).isEqualTo("mock exception");
}
}

@Test
Expand Down

0 comments on commit 35ad5c1

Please sign in to comment.