-
Notifications
You must be signed in to change notification settings - Fork 3.1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Always fire QueryCompletedEvent for DDL queries
Previously the event was fired because client protocol is reading the final query info. That is brittle and theoretically could be removed making DDL queries fail to trigger query completed event.
- Loading branch information
1 parent
21d3cc3
commit 3fe314c
Showing
2 changed files
with
96 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
90 changes: 90 additions & 0 deletions
90
testing/trino-tests/src/test/java/io/trino/execution/TestQueryCompletedEvent.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,90 @@ | ||
/* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package io.trino.execution; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import io.trino.dispatcher.DispatchQuery; | ||
import io.trino.plugin.memory.MemoryPlugin; | ||
import io.trino.spi.Plugin; | ||
import io.trino.spi.eventlistener.EventListener; | ||
import io.trino.spi.eventlistener.EventListenerFactory; | ||
import io.trino.spi.eventlistener.QueryCompletedEvent; | ||
import io.trino.testing.StandaloneQueryRunner; | ||
import io.trino.testing.TestingDirectTrinoClient; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.HashSet; | ||
import java.util.Map; | ||
import java.util.Set; | ||
|
||
import static io.trino.execution.QueryState.FINISHED; | ||
import static io.trino.testing.TestingSession.testSession; | ||
import static io.trino.testing.assertions.Assert.assertEventually; | ||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
public class TestQueryCompletedEvent | ||
{ | ||
@Test | ||
public void testQueryCompletedEventIssued() | ||
{ | ||
try (StandaloneQueryRunner queryRunner = new StandaloneQueryRunner(testSession())) { | ||
queryRunner.installPlugin(new MemoryPlugin()); | ||
queryRunner.createCatalog("memory", "memory"); | ||
Set<String> queryCompletedQueryIds = new HashSet<>(); | ||
EventListener listener = new EventListener() | ||
{ | ||
@Override | ||
public void queryCompleted(QueryCompletedEvent queryCompletedEvent) | ||
{ | ||
queryCompletedQueryIds.add(queryCompletedEvent.getMetadata().getQueryId()); | ||
} | ||
}; | ||
queryRunner.installPlugin(new Plugin() | ||
{ | ||
@Override | ||
public Iterable<EventListenerFactory> getEventListenerFactories() | ||
{ | ||
return ImmutableList.of(new EventListenerFactory() | ||
{ | ||
@Override | ||
public String getName() | ||
{ | ||
return "testQueryCompletedEventIssued"; | ||
} | ||
|
||
@Override | ||
public EventListener create(Map<String, String> config, EventListenerContext context) | ||
{ | ||
return listener; | ||
} | ||
}); | ||
} | ||
}); | ||
|
||
assertQueryCompletedIssued(queryRunner, queryCompletedQueryIds, "SELECT 1"); | ||
assertQueryCompletedIssued(queryRunner, queryCompletedQueryIds, "CREATE SCHEMA memory.test_schema"); | ||
assertQueryCompletedIssued(queryRunner, queryCompletedQueryIds, "DROP SCHEMA memory.test_schema"); | ||
} | ||
} | ||
|
||
private static void assertQueryCompletedIssued(StandaloneQueryRunner queryRunner, Set<String> queryCompletedQueryIds, String sql) | ||
{ | ||
TestingDirectTrinoClient.Result result = queryRunner.executeWithoutResults(testSession(), sql); | ||
DispatchQuery query = queryRunner.getCoordinator().getDispatchManager().getQuery(result.queryId()); | ||
assertThat(query.getState()).isEqualTo(FINISHED); | ||
assertEventually(() -> assertThat(queryCompletedQueryIds) | ||
.describedAs("query '%s'", sql) | ||
.contains(result.queryId().getId())); | ||
} | ||
} |