|
@@ -18,16 +18,13 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.timelineservice.aggregator;
|
|
|
|
|
|
-import java.net.URI;
|
|
|
import java.nio.ByteBuffer;
|
|
|
-import java.util.HashMap;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.http.lib.StaticUserWebFilter;
|
|
|
import org.apache.hadoop.util.ExitUtil;
|
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -35,144 +32,91 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
|
|
|
import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
|
|
|
import org.apache.hadoop.yarn.server.api.AuxiliaryService;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerContext;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
|
|
|
-import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
|
|
-import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
|
|
|
-import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
|
|
-import org.apache.hadoop.http.HttpServer2;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
-import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
|
|
|
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
|
|
|
-
|
|
|
/**
|
|
|
- * The top-level server for the per-node timeline aggregator service. Currently
|
|
|
+ * The top-level server for the per-node timeline aggregator collection. Currently
|
|
|
* it is defined as an auxiliary service to accommodate running within another
|
|
|
* daemon (e.g. node manager).
|
|
|
*/
|
|
|
@Private
|
|
|
@Unstable
|
|
|
-public class PerNodeAggregatorServer extends AuxiliaryService {
|
|
|
+public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
|
|
|
private static final Log LOG =
|
|
|
- LogFactory.getLog(PerNodeAggregatorServer.class);
|
|
|
+ LogFactory.getLog(PerNodeTimelineAggregatorsAuxService.class);
|
|
|
private static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
|
|
- static final String AGGREGATOR_COLLECTION_ATTR_KEY = "aggregator.collection";
|
|
|
|
|
|
- private final AppLevelServiceManager serviceManager;
|
|
|
- private HttpServer2 timelineRestServer;
|
|
|
+ private final TimelineAggregatorsCollection aggregatorCollection;
|
|
|
|
|
|
- public PerNodeAggregatorServer() {
|
|
|
+ public PerNodeTimelineAggregatorsAuxService() {
|
|
|
// use the same singleton
|
|
|
- this(AppLevelServiceManager.getInstance());
|
|
|
+ this(TimelineAggregatorsCollection.getInstance());
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- PerNodeAggregatorServer(AppLevelServiceManager serviceManager) {
|
|
|
+ @VisibleForTesting PerNodeTimelineAggregatorsAuxService(
|
|
|
+ TimelineAggregatorsCollection aggregatorCollection) {
|
|
|
super("timeline_aggregator");
|
|
|
- this.serviceManager = serviceManager;
|
|
|
+ this.aggregatorCollection = aggregatorCollection;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected void serviceInit(Configuration conf) throws Exception {
|
|
|
- serviceManager.init(conf);
|
|
|
+ aggregatorCollection.init(conf);
|
|
|
super.serviceInit(conf);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected void serviceStart() throws Exception {
|
|
|
+ aggregatorCollection.start();
|
|
|
super.serviceStart();
|
|
|
- serviceManager.start();
|
|
|
- startWebApp();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected void serviceStop() throws Exception {
|
|
|
- if (timelineRestServer != null) {
|
|
|
- timelineRestServer.stop();
|
|
|
- }
|
|
|
- // stop the service manager
|
|
|
- serviceManager.stop();
|
|
|
+ aggregatorCollection.stop();
|
|
|
super.serviceStop();
|
|
|
}
|
|
|
|
|
|
- private void startWebApp() {
|
|
|
- Configuration conf = getConfig();
|
|
|
- // use the same ports as the old ATS for now; we could create new properties
|
|
|
- // for the new timeline service if needed
|
|
|
- String bindAddress = WebAppUtils.getWebAppBindURL(conf,
|
|
|
- YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
|
|
|
- WebAppUtils.getAHSWebAppURLWithoutScheme(conf));
|
|
|
- LOG.info("Instantiating the per-node aggregator webapp at " + bindAddress);
|
|
|
- try {
|
|
|
- Configuration confForInfoServer = new Configuration(conf);
|
|
|
- confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
|
|
|
- HttpServer2.Builder builder = new HttpServer2.Builder()
|
|
|
- .setName("timeline")
|
|
|
- .setConf(conf)
|
|
|
- .addEndpoint(URI.create("http://" + bindAddress));
|
|
|
- timelineRestServer = builder.build();
|
|
|
- // TODO: replace this by an authentication filter in future.
|
|
|
- HashMap<String, String> options = new HashMap<>();
|
|
|
- String username = conf.get(HADOOP_HTTP_STATIC_USER,
|
|
|
- DEFAULT_HADOOP_HTTP_STATIC_USER);
|
|
|
- options.put(HADOOP_HTTP_STATIC_USER, username);
|
|
|
- HttpServer2.defineFilter(timelineRestServer.getWebAppContext(),
|
|
|
- "static_user_filter_timeline",
|
|
|
- StaticUserWebFilter.StaticUserFilter.class.getName(),
|
|
|
- options, new String[] {"/*"});
|
|
|
-
|
|
|
- timelineRestServer.addJerseyResourcePackage(
|
|
|
- PerNodeAggregatorWebService.class.getPackage().getName() + ";"
|
|
|
- + GenericExceptionHandler.class.getPackage().getName() + ";"
|
|
|
- + YarnJacksonJaxbJsonProvider.class.getPackage().getName(),
|
|
|
- "/*");
|
|
|
- timelineRestServer.setAttribute(AGGREGATOR_COLLECTION_ATTR_KEY,
|
|
|
- AppLevelServiceManager.getInstance());
|
|
|
- timelineRestServer.start();
|
|
|
- } catch (Exception e) {
|
|
|
- String msg = "The per-node aggregator webapp failed to start.";
|
|
|
- LOG.error(msg, e);
|
|
|
- throw new YarnRuntimeException(msg, e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
// these methods can be used as the basis for future service methods if the
|
|
|
// per-node aggregator runs separate from the node manager
|
|
|
/**
|
|
|
- * Creates and adds an app level aggregator service for the specified
|
|
|
- * application id. The service is also initialized and started. If the service
|
|
|
- * already exists, no new service is created.
|
|
|
+ * Creates and adds an app level aggregator for the specified application id.
|
|
|
+ * The aggregator is also initialized and started. If the service already
|
|
|
+ * exists, no new service is created.
|
|
|
*
|
|
|
* @return whether it was added successfully
|
|
|
*/
|
|
|
public boolean addApplication(ApplicationId appId) {
|
|
|
String appIdString = appId.toString();
|
|
|
- return serviceManager.addService(appIdString);
|
|
|
+ AppLevelTimelineAggregator aggregator =
|
|
|
+ new AppLevelTimelineAggregator(appIdString);
|
|
|
+ return (aggregatorCollection.putIfAbsent(appIdString, aggregator)
|
|
|
+ == aggregator);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Removes the app level aggregator service for the specified application id.
|
|
|
- * The service is also stopped as a result. If the service does not exist, no
|
|
|
+ * Removes the app level aggregator for the specified application id. The
|
|
|
+ * aggregator is also stopped as a result. If the aggregator does not exist, no
|
|
|
* change is made.
|
|
|
*
|
|
|
* @return whether it was removed successfully
|
|
|
*/
|
|
|
public boolean removeApplication(ApplicationId appId) {
|
|
|
String appIdString = appId.toString();
|
|
|
- return serviceManager.removeService(appIdString);
|
|
|
+ return aggregatorCollection.remove(appIdString);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Creates and adds an app level aggregator service for the specified
|
|
|
- * application id. The service is also initialized and started. If the service
|
|
|
- * already exists, no new service is created.
|
|
|
+ * Creates and adds an app level aggregator for the specified application id.
|
|
|
+ * The aggregator is also initialized and started. If the aggregator already
|
|
|
+ * exists, no new aggregator is created.
|
|
|
*/
|
|
|
@Override
|
|
|
public void initializeContainer(ContainerInitializationContext context) {
|
|
@@ -186,8 +130,8 @@ public class PerNodeAggregatorServer extends AuxiliaryService {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Removes the app level aggregator service for the specified application id.
|
|
|
- * The service is also stopped as a result. If the service does not exist, no
|
|
|
+ * Removes the app level aggregator for the specified application id. The
|
|
|
+ * aggregator is also stopped as a result. If the aggregator does not exist, no
|
|
|
* change is made.
|
|
|
*/
|
|
|
@Override
|
|
@@ -211,7 +155,7 @@ public class PerNodeAggregatorServer extends AuxiliaryService {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
boolean hasApplication(String appId) {
|
|
|
- return serviceManager.hasService(appId);
|
|
|
+ return aggregatorCollection.containsKey(appId);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -230,35 +174,35 @@ public class PerNodeAggregatorServer extends AuxiliaryService {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public static PerNodeAggregatorServer launchServer(String[] args) {
|
|
|
+ public static PerNodeTimelineAggregatorsAuxService launchServer(String[] args) {
|
|
|
Thread
|
|
|
.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
|
|
|
- StringUtils.startupShutdownMessage(PerNodeAggregatorServer.class, args,
|
|
|
+ StringUtils.startupShutdownMessage(PerNodeTimelineAggregatorsAuxService.class, args,
|
|
|
LOG);
|
|
|
- PerNodeAggregatorServer server = null;
|
|
|
+ PerNodeTimelineAggregatorsAuxService auxService = null;
|
|
|
try {
|
|
|
- server = new PerNodeAggregatorServer();
|
|
|
- ShutdownHookManager.get().addShutdownHook(new ShutdownHook(server),
|
|
|
+ auxService = new PerNodeTimelineAggregatorsAuxService();
|
|
|
+ ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService),
|
|
|
SHUTDOWN_HOOK_PRIORITY);
|
|
|
YarnConfiguration conf = new YarnConfiguration();
|
|
|
- server.init(conf);
|
|
|
- server.start();
|
|
|
+ auxService.init(conf);
|
|
|
+ auxService.start();
|
|
|
} catch (Throwable t) {
|
|
|
LOG.fatal("Error starting PerNodeAggregatorServer", t);
|
|
|
ExitUtil.terminate(-1, "Error starting PerNodeAggregatorServer");
|
|
|
}
|
|
|
- return server;
|
|
|
+ return auxService;
|
|
|
}
|
|
|
|
|
|
private static class ShutdownHook implements Runnable {
|
|
|
- private final PerNodeAggregatorServer server;
|
|
|
+ private final PerNodeTimelineAggregatorsAuxService auxService;
|
|
|
|
|
|
- public ShutdownHook(PerNodeAggregatorServer server) {
|
|
|
- this.server = server;
|
|
|
+ public ShutdownHook(PerNodeTimelineAggregatorsAuxService auxService) {
|
|
|
+ this.auxService = auxService;
|
|
|
}
|
|
|
|
|
|
public void run() {
|
|
|
- server.stop();
|
|
|
+ auxService.stop();
|
|
|
}
|
|
|
}
|
|
|
|