-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Add EventListener to HiveConnector #3358
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,7 @@ | |
| import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; | ||
| import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; | ||
| import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; | ||
| import io.prestosql.plugin.base.classloader.ClassLoaderSafeEventListener; | ||
| import io.prestosql.plugin.base.classloader.ClassLoaderSafeNodePartitioningProvider; | ||
| import io.prestosql.plugin.base.jmx.ConnectorObjectNameGeneratorModule; | ||
| import io.prestosql.plugin.base.jmx.MBeanServerModule; | ||
|
|
@@ -54,6 +55,7 @@ | |
| import io.prestosql.spi.connector.ConnectorPageSourceProvider; | ||
| import io.prestosql.spi.connector.ConnectorSplitManager; | ||
| import io.prestosql.spi.connector.SystemTable; | ||
| import io.prestosql.spi.eventlistener.EventListener; | ||
| import io.prestosql.spi.procedure.Procedure; | ||
| import io.prestosql.spi.type.TypeManager; | ||
| import org.weakref.jmx.guice.MBeanModule; | ||
|
|
@@ -62,6 +64,8 @@ | |
| import java.util.Optional; | ||
| import java.util.Set; | ||
|
|
||
| import static com.google.common.collect.ImmutableSet.toImmutableSet; | ||
| import static com.google.inject.multibindings.Multibinder.newSetBinder; | ||
| import static io.airlift.configuration.ConditionalModule.installModuleIf; | ||
| import static java.util.Objects.requireNonNull; | ||
|
|
||
|
|
@@ -104,6 +108,7 @@ public static Connector createConnector(String catalogName, Map<String, String> | |
| binder.bind(PageSorter.class).toInstance(context.getPageSorter()); | ||
| binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just add
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. better reading experience along with expressing the intention of it, so that the reader doesn't have to guess that it's a necessary default Event Listener.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is a matter of taste. I would prefer not to over-optimize reading experience, but rather follow the current layout.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Agreed, we should just bind an empty set here.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok |
||
| }, | ||
| binder -> newSetBinder(binder, EventListener.class), | ||
| module); | ||
|
|
||
| Injector injector = app | ||
|
|
@@ -133,6 +138,10 @@ public static Connector createConnector(String catalogName, Map<String, String> | |
| classLoader); | ||
| Set<Procedure> procedures = injector.getInstance(Key.get(new TypeLiteral<Set<Procedure>>() {})); | ||
| Set<SystemTable> systemTables = injector.getInstance(Key.get(new TypeLiteral<Set<SystemTable>>() {})); | ||
| Set<EventListener> eventListeners = injector.getInstance(Key.get(new TypeLiteral<Set<EventListener>>() {})) | ||
| .stream() | ||
| .map(listener -> new ClassLoaderSafeEventListener(listener, classLoader)) | ||
| .collect(toImmutableSet()); | ||
|
|
||
| return new HiveConnector( | ||
| lifeCycleManager, | ||
|
|
@@ -144,6 +153,7 @@ public static Connector createConnector(String catalogName, Map<String, String> | |
| new ClassLoaderSafeNodePartitioningProvider(connectorDistributionProvider, classLoader), | ||
| systemTables, | ||
| procedures, | ||
| eventListeners, | ||
| hiveSessionProperties.getSessionProperties(), | ||
| HiveSchemaProperties.SCHEMA_PROPERTIES, | ||
| hiveTableProperties.getTableProperties(), | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,59 @@ | ||
| /* | ||
| * 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.prestosql.plugin.base.classloader; | ||
|
|
||
| import io.prestosql.spi.classloader.ThreadContextClassLoader; | ||
| import io.prestosql.spi.eventlistener.EventListener; | ||
| import io.prestosql.spi.eventlistener.QueryCompletedEvent; | ||
| import io.prestosql.spi.eventlistener.QueryCreatedEvent; | ||
| import io.prestosql.spi.eventlistener.SplitCompletedEvent; | ||
|
|
||
| import static java.util.Objects.requireNonNull; | ||
|
|
||
| public class ClassLoaderSafeEventListener | ||
|
electrum marked this conversation as resolved.
Outdated
|
||
| implements EventListener | ||
| { | ||
| private final EventListener delegate; | ||
| private final ClassLoader classLoader; | ||
|
|
||
| public ClassLoaderSafeEventListener(@ForClassLoaderSafe EventListener delegate, ClassLoader classLoader) | ||
| { | ||
| this.delegate = requireNonNull(delegate, "delegate is null"); | ||
| this.classLoader = requireNonNull(classLoader, "classLoader is null"); | ||
| } | ||
|
|
||
| @Override | ||
| public void queryCreated(QueryCreatedEvent queryCreatedEvent) | ||
| { | ||
| try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { | ||
| delegate.queryCreated(queryCreatedEvent); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void queryCompleted(QueryCompletedEvent queryCompletedEvent) | ||
| { | ||
| try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { | ||
| delegate.queryCompleted(queryCompletedEvent); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void splitCompleted(SplitCompletedEvent splitCompletedEvent) | ||
| { | ||
| try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { | ||
| delegate.splitCompleted(splitCompletedEvent); | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.