Skip to content

Commit

Permalink
Add support for multiple event listeners
Browse files Browse the repository at this point in the history
  • Loading branch information
evanvdia committed Jan 29, 2025
1 parent e223cc1 commit 9e29e22
Show file tree
Hide file tree
Showing 11 changed files with 199 additions and 28 deletions.
11 changes: 11 additions & 0 deletions presto-docs/src/main/sphinx/develop/event-listener.rst
Original file line number Diff line number Diff line change
Expand Up @@ -46,3 +46,14 @@ Example configuration file:
event-listener.name=custom-event-listener
custom-property1=custom-value1
custom-property2=custom-value2
Multiple Event Listeners
------------------------

Multiple instances of the same, or different event listeners can be
installed and configured by setting ``event-listener.config-files``
to a comma separated list of config files.

.. code-block:: none
event-listener.config-files=etc/event-listener.properties,etc/event-listener-second.properties
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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 com.facebook.presto.eventlistener;

import com.facebook.airlift.configuration.Config;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;

import javax.validation.constraints.NotNull;

import java.io.File;
import java.util.List;

import static com.google.common.collect.ImmutableList.toImmutableList;

public class EventListenerConfig
{
private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings();
private List<File> eventListenerFiles = ImmutableList.of();

@NotNull
public List<File> getEventListenerFiles()
{
return eventListenerFiles;
}

@Config("event-listener.config-files")
public EventListenerConfig setEventListenerFiles(String eventListenerFiles)
{
this.eventListenerFiles = SPLITTER.splitToList(eventListenerFiles).stream()
.map(File::new)
.collect(toImmutableList());
return this;
}

public EventListenerConfig setEventListenerFiles(List<File> eventListenerFiles)
{
this.eventListenerFiles = ImmutableList.copyOf(eventListenerFiles);
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,17 @@
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;

import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;

Expand All @@ -44,9 +49,16 @@ public class EventListenerManager
private static final Logger log = Logger.get(EventListenerManager.class);
private static final File EVENT_LISTENER_CONFIGURATION = new File("etc/event-listener.properties");
private static final String EVENT_LISTENER_PROPERTY_NAME = "event-listener.name";

private final List<File> configFiles;
private final Map<String, EventListenerFactory> eventListenerFactories = new ConcurrentHashMap<>();
private final AtomicReference<Optional<EventListener>> configuredEventListener = new AtomicReference<>(Optional.empty());
private final AtomicReference<List<EventListener>> configuredEventListeners =
new AtomicReference<>(ImmutableList.of());

@Inject
public EventListenerManager(EventListenerConfig config)
{
this.configFiles = ImmutableList.copyOf(config.getEventListenerFiles());
}

public void addEventListenerFactory(EventListenerFactory eventListenerFactory)
{
Expand All @@ -57,17 +69,37 @@ public void addEventListenerFactory(EventListenerFactory eventListenerFactory)
}
}

public void loadConfiguredEventListener()
throws Exception
public void loadConfiguredEventListeners()
{
if (EVENT_LISTENER_CONFIGURATION.exists()) {
Map<String, String> properties = loadProperties(EVENT_LISTENER_CONFIGURATION);
checkArgument(
!isNullOrEmpty(properties.get(EVENT_LISTENER_PROPERTY_NAME)),
"Access control configuration %s does not contain %s",
EVENT_LISTENER_CONFIGURATION.getAbsoluteFile(),
EVENT_LISTENER_PROPERTY_NAME);
loadConfiguredEventListener(properties);
List<File> configFiles = this.configFiles;
if (configFiles.isEmpty()) {
if (!EVENT_LISTENER_CONFIGURATION.exists()) {
return;
}
configFiles = ImmutableList.of(EVENT_LISTENER_CONFIGURATION);
}
configFiles.forEach(this::createEventListener);
}

private void createEventListener(File configFile)
{
log.info("-- Loading event listener %s --", configFile);
if (configFile.exists()) {
configFile = configFile.getAbsoluteFile();
log.info("-- Loading event listener configFile.getAbsoluteFile() : %s --", configFile);
Map<String, String> properties;
try {
properties = new HashMap<>(loadProperties(configFile));
loadConfiguredEventListener(properties);
log.info("-- Loaded event listener %s --", configFile);
}
catch (IOException e) {
log.error(e, "IOException while loading configuration file: " + configFile);
throw new UncheckedIOException("Failed to read configuration file: " + configFile, e);
}
}
else {
log.info("Unable to locate ConfigFile %s --", configFile);
}
}

Expand All @@ -92,39 +124,41 @@ protected void setConfiguredEventListener(String name, Map<String, String> prope

try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(eventListenerFactory.getClass().getClassLoader())) {
EventListener eventListener = eventListenerFactory.create(ImmutableMap.copyOf(properties));
this.configuredEventListener.set(Optional.of(eventListener));
List<EventListener> eventListeners = new ArrayList<>(this.configuredEventListeners.get());
eventListeners.add(eventListener);
this.configuredEventListeners.set(eventListeners);
}

log.info("-- Loaded event listener %s --", name);
}

public void queryCompleted(QueryCompletedEvent queryCompletedEvent)
{
configuredEventListener.get()
.ifPresent(eventListener -> eventListener.queryCompleted(queryCompletedEvent));
configuredEventListeners.get()
.forEach(eventListener -> eventListener.queryCompleted(queryCompletedEvent));
}

public void queryCreated(QueryCreatedEvent queryCreatedEvent)
{
configuredEventListener.get()
.ifPresent(eventListener -> eventListener.queryCreated(queryCreatedEvent));
configuredEventListeners.get()
.forEach(eventListener -> eventListener.queryCreated(queryCreatedEvent));
}

public void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
{
configuredEventListener.get()
.ifPresent(eventListener -> eventListener.queryUpdated(queryUpdatedEvent));
configuredEventListeners.get()
.forEach(eventListener -> eventListener.queryUpdated(queryUpdatedEvent));
}

public void publishQueryProgress(QueryProgressEvent queryProgressEvent)
{
configuredEventListener.get()
.ifPresent(eventListener -> eventListener.publishQueryProgress(queryProgressEvent));
configuredEventListeners.get()
.forEach(eventListener -> eventListener.publishQueryProgress(queryProgressEvent));
}

public void splitCompleted(SplitCompletedEvent splitCompletedEvent)
{
configuredEventListener.get()
.ifPresent(eventListener -> eventListener.splitCompleted(splitCompletedEvent));
configuredEventListeners.get()
.forEach(eventListener -> eventListener.splitCompleted(splitCompletedEvent));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,17 @@
import com.google.inject.Module;
import com.google.inject.Scopes;

import static com.facebook.airlift.configuration.ConfigBinder.configBinder;
import static org.weakref.jmx.guice.ExportBinder.newExporter;

public class EventListenerModule
implements Module
{
@Override
public void configure(Binder binder)
{
configBinder(binder).bindConfig(EventListenerConfig.class);
binder.bind(EventListenerManager.class).in(Scopes.SINGLETON);
newExporter(binder).export(EventListenerManager.class).withGeneratedName();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ public void run()
}
injector.getInstance(PasswordAuthenticatorManager.class).loadPasswordAuthenticator();
injector.getInstance(PrestoAuthenticatorManager.class).loadPrestoAuthenticator();
injector.getInstance(EventListenerManager.class).loadConfiguredEventListener();
injector.getInstance(EventListenerManager.class).loadConfiguredEventListeners();
injector.getInstance(TempStorageManager.class).loadTempStorages();
injector.getInstance(QueryPrerequisitesManager.class).loadQueryPrerequisites();
injector.getInstance(NodeTtlFetcherManager.class).loadNodeTtlFetcher();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import com.facebook.presto.cost.StatsCalculator;
import com.facebook.presto.dispatcher.DispatchManager;
import com.facebook.presto.dispatcher.QueryPrerequisitesManagerModule;
import com.facebook.presto.eventlistener.EventListenerConfig;
import com.facebook.presto.eventlistener.EventListenerManager;
import com.facebook.presto.execution.QueryInfo;
import com.facebook.presto.execution.QueryManager;
Expand Down Expand Up @@ -322,6 +323,7 @@ public TestingPrestoServer(
binder.bind(TestingTempStorageManager.class).in(Scopes.SINGLETON);
binder.bind(AccessControlManager.class).to(TestingAccessControlManager.class).in(Scopes.SINGLETON);
binder.bind(EventListenerManager.class).to(TestingEventListenerManager.class).in(Scopes.SINGLETON);
binder.bind(EventListenerConfig.class).in(Scopes.SINGLETON);
binder.bind(TempStorageManager.class).to(TestingTempStorageManager.class).in(Scopes.SINGLETON);
binder.bind(AccessControl.class).to(AccessControlManager.class).in(Scopes.SINGLETON);
binder.bind(ShutdownAction.class).to(TestShutdownAction.class).in(Scopes.SINGLETON);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import com.facebook.presto.cost.TaskCountEstimator;
import com.facebook.presto.dispatcher.NoOpQueryManager;
import com.facebook.presto.dispatcher.QueryPrerequisitesManager;
import com.facebook.presto.eventlistener.EventListenerConfig;
import com.facebook.presto.eventlistener.EventListenerManager;
import com.facebook.presto.execution.AlterFunctionTask;
import com.facebook.presto.execution.CommitTask;
Expand Down Expand Up @@ -525,7 +526,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
accessControl,
new PasswordAuthenticatorManager(),
new PrestoAuthenticatorManager(new SecurityConfig()),
new EventListenerManager(),
new EventListenerManager(new EventListenerConfig()),
blockEncodingManager,
new TestingTempStorageManager(),
new QueryPrerequisitesManager(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.testing;

import com.facebook.presto.eventlistener.EventListenerConfig;
import com.facebook.presto.eventlistener.EventListenerManager;
import com.facebook.presto.spi.eventlistener.EventListener;
import com.facebook.presto.spi.eventlistener.EventListenerFactory;
Expand All @@ -22,6 +23,7 @@
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;

import java.util.Optional;
import java.util.concurrent.atomic.AtomicReference;
Expand All @@ -31,6 +33,12 @@ public class TestingEventListenerManager
{
private final AtomicReference<Optional<EventListener>> configuredEventListener = new AtomicReference<>(Optional.empty());

@Inject
public TestingEventListenerManager(EventListenerConfig config)
{
super(config);
}

@Override
public void addEventListenerFactory(EventListenerFactory eventListenerFactory)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.facebook.presto.cost.HistoryBasedPlanStatisticsManager;
import com.facebook.presto.event.QueryMonitor;
import com.facebook.presto.event.QueryMonitorConfig;
import com.facebook.presto.eventlistener.EventListenerConfig;
import com.facebook.presto.eventlistener.EventListenerManager;
import com.facebook.presto.execution.ClusterSizeMonitor;
import com.facebook.presto.execution.ExecutionFailureInfo;
Expand Down Expand Up @@ -464,7 +465,7 @@ private QueryMonitor createQueryMonitor(CountingEventListener eventListener)

private EventListenerManager createEventListenerManager(CountingEventListener countingEventListener)
{
EventListenerManager eventListenerManager = new EventListenerManager();
EventListenerManager eventListenerManager = new EventListenerManager(new EventListenerConfig());
eventListenerManager.addEventListenerFactory(new TestEventListenerFactory(countingEventListener));
eventListenerManager.loadConfiguredEventListener(ImmutableMap.of("event-listener.name", TestEventListenerFactory.NAME));
return eventListenerManager;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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 com.facebook.presto.eventlistener;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.testng.annotations.Test;

import java.io.File;
import java.util.Map;

import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping;
import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults;
import static com.facebook.airlift.configuration.testing.ConfigAssertions.recordDefaults;
public class TestEventListenerConfig
{
@Test
public void testDefaults()
{
assertRecordedDefaults(recordDefaults(EventListenerConfig.class)
.setEventListenerFiles(""));
}

@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
.put("event-listener.config-files", "a,b,c")
.build();

EventListenerConfig expected = new EventListenerConfig()
.setEventListenerFiles("a,b,c");
assertFullMapping(properties, expected);

ImmutableList.Builder<File> filesBuilder = ImmutableList.builder();
filesBuilder.add(new File("a"), new File("b"), new File("c"));
//Test List version
expected = new EventListenerConfig()
.setEventListenerFiles(filesBuilder.build());

assertFullMapping(properties, expected);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.facebook.presto.cost.StatsAndCosts;
import com.facebook.presto.dispatcher.NoOpQueryManager;
import com.facebook.presto.event.SplitMonitor;
import com.facebook.presto.eventlistener.EventListenerConfig;
import com.facebook.presto.eventlistener.EventListenerManager;
import com.facebook.presto.execution.buffer.OutputBuffers;
import com.facebook.presto.execution.scheduler.LegacyNetworkTopology;
Expand Down Expand Up @@ -201,7 +202,7 @@ public static TaskInfo updateTask(SqlTask sqlTask, List<TaskSource> taskSources,
public static SplitMonitor createTestSplitMonitor()
{
return new SplitMonitor(
new EventListenerManager(),
new EventListenerManager(new EventListenerConfig()),
new JsonObjectMapperProvider().get());
}

Expand Down

0 comments on commit 9e29e22

Please sign in to comment.