diff --git a/presto-main/src/main/java/com/facebook/presto/application/ApplicationManager.java b/presto-main/src/main/java/com/facebook/presto/application/ApplicationManager.java new file mode 100644 index 000000000000..f79ae447c492 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/application/ApplicationManager.java @@ -0,0 +1,114 @@ +/* + * 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.application; + +import com.facebook.presto.spi.application.Application; +import com.facebook.presto.spi.application.ApplicationFactory; +import com.facebook.presto.spi.classloader.ThreadContextClassLoader; +import com.google.inject.Injector; +import io.airlift.log.Logger; + +import javax.annotation.PreDestroy; +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +@ThreadSafe +public class ApplicationManager +{ + private static final Logger log = Logger.get(ApplicationManager.class); + + @GuardedBy("this") + private final Map applicationFactories = new ConcurrentHashMap<>(); + @GuardedBy("this") + private final Map applications = new ConcurrentHashMap<>(); + + private final AtomicBoolean stopped = new AtomicBoolean(); + + public synchronized void addApplicationFactory(ApplicationFactory applicationFactory) + { + requireNonNull(applicationFactory, "applicationFactory is null"); + + if (applicationFactories.putIfAbsent(applicationFactory.getName(), applicationFactory) != null) { + throw new IllegalArgumentException(format("Application '%s' is already registered", applicationFactory.getName())); + } + } + + @PreDestroy + public void stop() + { + if (stopped.getAndSet(true)) { + return; + } + + for (Map.Entry entry : applications.entrySet()) { + Application application = entry.getValue(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(application.getClass().getClassLoader())) { + application.shutdown(); + } + catch (Throwable t) { + log.error(t, "Error shutting down application: %s", entry.getKey()); + } + } + } + + protected synchronized void setup(String applicationName, ApplicationFactory applicationFactory, Map properties) + { + checkState(!stopped.get(), "ApplicationManager is stopped"); + checkState(!applications.containsKey(applicationName), "Application %s already exists", applicationName); + try { + applications.put(applicationName, applicationFactory.create(properties)); + } + catch (Throwable t) { + log.error(t, "Error initializing application: %s", applicationFactory.getName()); + } + } + + public void setup(String applicationName, Map properties) + { + checkState(!stopped.get(), "ApplicationManager is stopped"); + ApplicationFactory applicationFactory = applicationFactories.get(applicationName); + checkArgument(applicationFactory != null, "No factory for application %s", applicationName); + setup(applicationName, applicationFactory, properties); + } + + protected void start(Injector injector) + { + for (Map.Entry entry : applications.entrySet()) { + Application application = entry.getValue(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(application.getClass().getClassLoader())) { + application.run(injector); + } + catch (Throwable t) { + log.error(t, "Error starting application: %s", entry.getKey()); + } + } + } + + public void startApplications(Injector injector) + { + requireNonNull(injector, "injector is null"); + checkState(!stopped.get(), "ApplicationManager is stopped"); + start(injector); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/application/ApplicationModule.java b/presto-main/src/main/java/com/facebook/presto/application/ApplicationModule.java new file mode 100644 index 000000000000..0ddc4e37fb8a --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/application/ApplicationModule.java @@ -0,0 +1,30 @@ +/* + * 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.application; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +public class ApplicationModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(ApplicationManager.class).in(Scopes.SINGLETON); + binder.bind(StaticApplicationStore.class).in(Scopes.SINGLETON); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/application/StaticApplicationStore.java b/presto-main/src/main/java/com/facebook/presto/application/StaticApplicationStore.java new file mode 100644 index 000000000000..985d68e4ff35 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/application/StaticApplicationStore.java @@ -0,0 +1,104 @@ +/* + * 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.application; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import java.io.File; +import java.io.FileInputStream; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.Maps.fromProperties; +import static java.util.Objects.requireNonNull; + +public class StaticApplicationStore +{ + private static final Logger log = Logger.get(StaticApplicationStore.class); + private static final File APPLICATION_CONFIGURATION_DIR = new File("etc/application/"); + private final ApplicationManager applicationManager; + private final AtomicBoolean applicationsLoading = new AtomicBoolean(); + private final AtomicBoolean applicationsLoaded = new AtomicBoolean(); + + @Inject + public StaticApplicationStore(ApplicationManager applicationManager) + { + this.applicationManager = requireNonNull(applicationManager, "applicationManager is null"); + } + + public boolean areApplicationsLoaded() + { + return applicationsLoaded.get(); + } + + public void loadApplications() + throws Exception + { + if (!applicationsLoading.compareAndSet(false, true)) { + return; + } + + for (File file : listFiles(APPLICATION_CONFIGURATION_DIR)) { + if (file.isFile() && file.getName().endsWith(".properties")) { + loadApplication(file); + } + } + + applicationsLoaded.set(true); + } + + private void loadApplication(File file) + throws Exception + { + log.info("-- Loading application %s --", file); + Map properties = new HashMap<>(loadProperties(file)); + + String applicationName = properties.remove("application.name"); + checkState(applicationName != null, "Application configuration %s does not contain application.name", file.getAbsoluteFile()); + + applicationManager.setup(applicationName, ImmutableMap.copyOf(properties)); + log.info("-- Added application %s --", applicationName); + } + + private static List listFiles(File installedPluginsDir) + { + if (installedPluginsDir != null && installedPluginsDir.isDirectory()) { + File[] files = installedPluginsDir.listFiles(); + if (files != null) { + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private static Map loadProperties(File file) + throws Exception + { + requireNonNull(file, "file is null"); + + Properties properties = new Properties(); + try (FileInputStream in = new FileInputStream(file)) { + properties.load(in); + } + return fromProperties(properties); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java b/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java index dd9b6830b58b..362eac7f4c8c 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.server; +import com.facebook.presto.application.ApplicationManager; import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.connector.ConnectorManager; import com.facebook.presto.eventlistener.EventListenerManager; @@ -20,6 +21,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.security.AccessControlManager; import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.application.ApplicationFactory; import com.facebook.presto.spi.block.BlockEncodingFactory; import com.facebook.presto.spi.classloader.ThreadContextClassLoader; import com.facebook.presto.spi.connector.ConnectorFactory; @@ -73,6 +75,7 @@ public class PluginManager private final ConnectorManager connectorManager; private final Metadata metadata; + private final ApplicationManager applicationManager; private final ResourceGroupManager resourceGroupManager; private final AccessControlManager accessControlManager; private final EventListenerManager eventListenerManager; @@ -91,6 +94,7 @@ public PluginManager( PluginManagerConfig config, ConnectorManager connectorManager, Metadata metadata, + ApplicationManager applicationManager, ResourceGroupManager resourceGroupManager, AccessControlManager accessControlManager, EventListenerManager eventListenerManager, @@ -112,6 +116,7 @@ public PluginManager( this.connectorManager = requireNonNull(connectorManager, "connectorManager is null"); this.metadata = requireNonNull(metadata, "metadata is null"); + this.applicationManager = requireNonNull(applicationManager, "applicationManager is null"); this.resourceGroupManager = requireNonNull(resourceGroupManager, "resourceGroupManager is null"); this.accessControlManager = requireNonNull(accessControlManager, "accessControlManager is null"); this.eventListenerManager = requireNonNull(eventListenerManager, "eventListenerManager is null"); @@ -209,6 +214,11 @@ public void installPlugin(Plugin plugin) log.info("Registering event listener %s", eventListenerFactory.getName()); eventListenerManager.addEventListenerFactory(eventListenerFactory); } + + for (ApplicationFactory applicationFactory : plugin.getApplicationFactories()) { + log.info("Registering application %s", applicationFactory.getName()); + applicationManager.addApplicationFactory(applicationFactory); + } } private URLClassLoader buildClassLoader(String plugin) diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index cfa456363ce7..673920612531 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.server; +import com.facebook.presto.application.ApplicationManager; +import com.facebook.presto.application.ApplicationModule; +import com.facebook.presto.application.StaticApplicationStore; import com.facebook.presto.discovery.EmbeddedDiscoveryModule; import com.facebook.presto.eventlistener.EventListenerManager; import com.facebook.presto.eventlistener.EventListenerModule; @@ -105,6 +108,7 @@ public void run() new ServerSecurityModule(), new AccessControlModule(), new EventListenerModule(), + new ApplicationModule(), new ServerMainModule(sqlParserOptions), new GracefulShutdownModule()); @@ -119,6 +123,9 @@ public void run() injector.getInstance(StaticCatalogStore.class).loadCatalogs(); + injector.getInstance(StaticApplicationStore.class).loadApplications(); + injector.getInstance(ApplicationManager.class).startApplications(injector); + // TODO: remove this huge hack updateConnectorIds( injector.getInstance(Announcer.class), diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java b/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java index 125002a028f4..d36b8cff40fa 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.spi; +import com.facebook.presto.spi.application.ApplicationFactory; import com.facebook.presto.spi.block.BlockEncodingFactory; import com.facebook.presto.spi.block.BlockEncodingSerde; import com.facebook.presto.spi.connector.ConnectorFactory; @@ -68,4 +69,9 @@ default Iterable getResourceGroupConfi { return emptyList(); } + + default Iterable getApplicationFactories() + { + return emptyList(); + } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/application/Application.java b/presto-spi/src/main/java/com/facebook/presto/spi/application/Application.java new file mode 100644 index 000000000000..3490427d34bb --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/application/Application.java @@ -0,0 +1,28 @@ +/* + * 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.spi.application; + +public interface Application + extends Runnable +{ + default void run(Object context) + { + run(); + } + + default void shutdown() + { + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/application/ApplicationFactory.java b/presto-spi/src/main/java/com/facebook/presto/spi/application/ApplicationFactory.java new file mode 100644 index 000000000000..1af0bf2b5fd9 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/application/ApplicationFactory.java @@ -0,0 +1,23 @@ +/* + * 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.spi.application; + +import java.util.Map; + +public interface ApplicationFactory +{ + String getName(); + + Application create(Map config); +}