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

Update tests for multiple eventlistener plugin changes #24755

Merged
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 @@ -24,7 +24,6 @@
import org.testng.annotations.AfterClass;
import org.testng.annotations.Test;

import java.util.Optional;
import java.util.Set;

import static com.facebook.presto.SystemSessionProperties.CTE_MATERIALIZATION_STRATEGY;
Expand All @@ -36,6 +35,7 @@
import static java.lang.String.format;
import static java.util.stream.Collectors.joining;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;

@Test(singleThreaded = true)
Expand All @@ -60,10 +60,10 @@ protected boolean supportsNotNullColumns()
public void close()
throws Exception
{
Optional<EventListener> eventListener = getQueryRunner().getEventListener();
assertTrue(eventListener.isPresent());
assertTrue(eventListener.get() instanceof TestingHiveEventListener, eventListener.get().getClass().getName());
Set<QueryId> runningQueryIds = ((TestingHiveEventListener) eventListener.get()).getRunningQueries();
assertFalse(getQueryRunner().getEventListeners().isEmpty());
EventListener eventListener = getQueryRunner().getEventListeners().get(0);
assertTrue(eventListener instanceof TestingHiveEventListener, eventListener.getClass().getName());
Set<QueryId> runningQueryIds = ((TestingHiveEventListener) eventListener).getRunningQueries();

if (!runningQueryIds.isEmpty()) {
// Await query events to propagate and finish
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@

import static com.facebook.presto.hive.HiveSessionProperties.DYNAMIC_SPLIT_SIZES_ENABLED;
import static io.airlift.tpch.TpchTable.ORDERS;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;

@Test(singleThreaded = true)
Expand Down Expand Up @@ -72,10 +73,10 @@ public void testDynamicSplits()

private TestHiveEventListenerPlugin.TestingHiveEventListener getEventListener()
{
Optional<EventListener> eventListener = getQueryRunner().getEventListener();
assertTrue(eventListener.isPresent());
assertTrue(eventListener.get() instanceof TestHiveEventListenerPlugin.TestingHiveEventListener, eventListener.get().getClass().getName());
return (TestHiveEventListenerPlugin.TestingHiveEventListener) eventListener.get();
assertFalse(getQueryRunner().getEventListeners().isEmpty());
EventListener eventListener = getQueryRunner().getEventListeners().get(0);
assertTrue(eventListener instanceof TestHiveEventListenerPlugin.TestingHiveEventListener, eventListener.getClass().getName());
return (TestHiveEventListenerPlugin.TestingHiveEventListener) eventListener;
}

private Session dynamicSplitsSession()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -628,10 +628,10 @@ public StatsCalculator getStatsCalculator()
return statsCalculator;
}

public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
checkState(coordinator, "not a coordinator");
return eventListenerManager.getEventListener();
return eventListenerManager.getEventListeners();
}

public TestingAccessControlManager getAccessControl()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -713,9 +713,9 @@ public StatsCalculator getStatsCalculator()
}

@Override
public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
return Optional.empty();
return ImmutableList.of();
}

public CostCalculator getCostCalculator()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,16 @@ public interface QueryRunner

StatsCalculator getStatsCalculator();

Optional<EventListener> getEventListener();
@Deprecated
default Optional<EventListener> getEventListener()
{
if (getEventListeners().isEmpty()) {
return Optional.empty();
}
return Optional.of(getEventListeners().get(0));
}

List<EventListener> getEventListeners();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we add a log here if there are multiple event listeners?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Adding a log message here, is not a good convention. What I could do is add an assert, that invoker of this method expects no more than one eventlistener. But that has consequences, if people are upgrading.


TestingAccessControlManager getAccessControl();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,17 @@
import com.facebook.presto.spi.eventlistener.QueryProgressEvent;
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;

import java.util.Optional;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;

public class TestingEventListenerManager
extends EventListenerManager
{
private final AtomicReference<Optional<EventListener>> configuredEventListener = new AtomicReference<>(Optional.empty());
private final AtomicReference<List<EventListener>> configuredEventListeners = new AtomicReference<>(ImmutableList.of());

@Inject
public TestingEventListenerManager(EventListenerConfig config)
Expand All @@ -42,51 +43,46 @@ public TestingEventListenerManager(EventListenerConfig config)
@Override
public void addEventListenerFactory(EventListenerFactory eventListenerFactory)
{
configuredEventListener.set(Optional.of(eventListenerFactory.create(ImmutableMap.of())));
configuredEventListeners.set(ImmutableList.of(eventListenerFactory.create(ImmutableMap.of())));
}

@Override
public void queryCompleted(QueryCompletedEvent queryCompletedEvent)
{
if (configuredEventListener.get().isPresent()) {
configuredEventListener.get().get().queryCompleted(queryCompletedEvent);
}
configuredEventListeners.get()
.forEach(eventListener -> eventListener.queryCompleted(queryCompletedEvent));
}

@Override
public void queryCreated(QueryCreatedEvent queryCreatedEvent)
{
if (configuredEventListener.get().isPresent()) {
configuredEventListener.get().get().queryCreated(queryCreatedEvent);
}
configuredEventListeners.get()
.forEach(eventListener -> eventListener.queryCreated(queryCreatedEvent));
}

@Override
public void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
{
if (configuredEventListener.get().isPresent()) {
configuredEventListener.get().get().queryUpdated(queryUpdatedEvent);
}
configuredEventListeners.get()
.forEach(eventListener -> eventListener.queryUpdated(queryUpdatedEvent));
}

@Override
public void publishQueryProgress(QueryProgressEvent queryProgressEvent)
{
if (configuredEventListener.get().isPresent()) {
configuredEventListener.get().get().publishQueryProgress(queryProgressEvent);
}
configuredEventListeners.get()
.forEach(eventListener -> eventListener.publishQueryProgress(queryProgressEvent));
}

@Override
public void splitCompleted(SplitCompletedEvent splitCompletedEvent)
{
if (configuredEventListener.get().isPresent()) {
configuredEventListener.get().get().splitCompleted(splitCompletedEvent);
}
configuredEventListeners.get()
.forEach(eventListener -> eventListener.splitCompleted(splitCompletedEvent));
}

public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
return configuredEventListener.get();
return configuredEventListeners.get();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.logging.Logger;
Expand Down Expand Up @@ -223,7 +222,7 @@ public StatsCalculator getStatsCalculator()
}

@Override
public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
throw new UnsupportedOperationException();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -498,9 +498,9 @@ public StatsCalculator getStatsCalculator()
}

@Override
public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
return Optional.empty();
return ImmutableList.of();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -607,10 +607,10 @@ public StatsCalculator getStatsCalculator()
}

@Override
public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this should be Optional<List

Copy link
Contributor Author

@ScrapCodes ScrapCodes Mar 21, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @jaystarshot for taking a look. I am not fully sure, I understand that. List can have zero elements (an List#isEmpty check similar to Optional#isPresent check), and if we are concerned about not breaking existing users, we added a default Optional<EventListener> in QueryRunner interface Link.

{
checkState(coordinators.size() == 1, "Expected a single coordinator");
return coordinators.get(0).getEventListener();
return coordinators.get(0).getEventListeners();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@

import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
Expand Down Expand Up @@ -178,9 +177,9 @@ public StatsCalculator getStatsCalculator()
}

@Override
public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
return server.getEventListener();
return server.getEventListeners();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.locks.Lock;

import static com.facebook.airlift.testing.Closeables.closeQuietly;
Expand Down Expand Up @@ -244,9 +243,9 @@ public StatsCalculator getStatsCalculator()
}

@Override
public Optional<EventListener> getEventListener()
public List<EventListener> getEventListeners()
{
return source.getEventListener();
return source.getEventListeners();
}

@Override
Expand Down
Loading