Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Always fire QueryCompletedEvent for DDL queries #24651

Merged
merged 2 commits into from
Jan 10, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,12 @@ private DataDefinitionExecution(
this.stateMachine = requireNonNull(stateMachine, "stateMachine is null");
this.parameters = parameters;
this.warningCollector = requireNonNull(warningCollector, "warningCollector is null");
stateMachine.addStateChangeListener(state -> {
if (state.isDone() && stateMachine.getFinalQueryInfo().isEmpty()) {
// make sure the final query info is set and listeners are triggered
stateMachine.updateQueryInfo(Optional.empty());
}
});
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,14 +105,20 @@ public List<SpanData> getSpans()
@Override
public MaterializedResult execute(Session session, @Language("SQL") String sql)
{
return executeInternal(session, sql).result();
return executeInternal(session, sql).result().get();
}

@Override
public MaterializedResultWithPlan executeWithPlan(Session session, String sql)
{
TestingDirectTrinoClient.Result result = executeInternal(session, sql);
return new MaterializedResultWithPlan(result.queryId(), server.getQueryPlan(result.queryId()), result.result());
MaterializedResult materializedRows = result.result().get();
return new MaterializedResultWithPlan(result.queryId(), server.getQueryPlan(result.queryId()), materializedRows);
}

public TestingDirectTrinoClient.Result executeWithoutResults(Session session, String sql)
{
return executeInternal(session, sql);
}

private TestingDirectTrinoClient.Result executeInternal(Session session, @Language("SQL") String sql)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.util.List;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.function.Supplier;

import static com.google.common.base.Preconditions.checkArgument;
import static io.trino.execution.QueryState.FINISHED;
Expand All @@ -63,7 +64,7 @@ public Result execute(SessionContext sessionContext, @Language("SQL") String sql
{
MaterializedQueryResultsListener queryResultsListener = new MaterializedQueryResultsListener();
DispatchQuery dispatchQuery = directTrinoClient.execute(sessionContext, sql, queryResultsListener);
return new Result(dispatchQuery.getQueryId(), toMaterializedRows(dispatchQuery, queryResultsListener.columnTypes(), queryResultsListener.columnNames(), queryResultsListener.pages()));
return new Result(dispatchQuery.getQueryId(), () -> toMaterializedRows(dispatchQuery, queryResultsListener.columnTypes(), queryResultsListener.columnNames(), queryResultsListener.pages()));
}

private static MaterializedResult toMaterializedRows(DispatchQuery dispatchQuery, List<Type> columnTypes, List<String> columnNames, List<Page> pages)
Expand Down Expand Up @@ -137,9 +138,9 @@ private static List<MaterializedRow> toMaterializedRows(ConnectorSession session
return rows.build();
}

record Result(QueryId queryId, MaterializedResult result)
public record Result(QueryId queryId, Supplier<MaterializedResult> result)
{
Result
public Result
{
requireNonNull(queryId, "queryId is null");
requireNonNull(result, "result is null");
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* 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 java.util.Collections.synchronizedSet;
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 = synchronizedSet(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()));
}
}
Loading