Browse Source

YARN-3210. Refactored timeline aggregator according to new code organization proposed in YARN-3166. Contributed by Li Lu.

(cherry picked from commit d3ff7f06cbc66d3a23c2551e7d4c752689f46afe)
Zhijie Shen 10 years ago
parent
commit
bf54d32750
15 changed files with 326 additions and 349 deletions
  1. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  2. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
  3. 0 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java
  4. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
  5. 0 136
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelServiceManager.java
  6. 0 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelServiceManagerProvider.java
  7. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelTimelineAggregator.java
  8. 40 96
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java
  9. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregator.java
  10. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorWebService.java
  11. 203 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java
  12. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestAppLevelTimelineAggregator.java
  13. 40 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeTimelineAggregatorsAuxService.java
  14. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregator.java
  15. 18 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -78,7 +78,7 @@ import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
 import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeAggregatorServer;
+import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeTimelineAggregatorsAuxService;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -159,7 +159,7 @@ public class TestDistributedShell {
       // enable aux-service based timeline aggregators
       conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
       conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
-        + ".class", PerNodeAggregatorServer.class.getName());
+        + ".class", PerNodeTimelineAggregatorsAuxService.class.getName());
     } else {
       Assert.fail("Wrong timeline version number: " + timelineVersion);
     }

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml

@@ -52,11 +52,6 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-api</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
-      <version>${project.version}</version>
-    </dependency>
     <dependency>
       <groupId>javax.xml.bind</groupId>
       <artifactId>jaxb-api</artifactId>

+ 0 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java

@@ -30,9 +30,6 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
-import org.apache.hadoop.yarn.server.timelineservice.aggregator.AppLevelServiceManager;
-import org.apache.hadoop.yarn.server.timelineservice.aggregator.AppLevelServiceManagerProvider;
-import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeAggregatorWebService;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java

@@ -6,7 +6,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeAggregatorServer;
+import org.apache.hadoop.yarn.server.timelineservice.aggregator.PerNodeTimelineAggregatorsAuxService;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -14,13 +14,13 @@ import org.junit.Test;
 import static org.junit.Assert.fail;
 
 public class TestTimelineServiceClientIntegration {
-  private static PerNodeAggregatorServer server;
+  private static PerNodeTimelineAggregatorsAuxService auxService;
 
   @BeforeClass
   public static void setupClass() throws Exception {
     try {
-      server = PerNodeAggregatorServer.launchServer(new String[0]);
-      server.addApplication(ApplicationId.newInstance(0, 1));
+      auxService = PerNodeTimelineAggregatorsAuxService.launchServer(new String[0]);
+      auxService.addApplication(ApplicationId.newInstance(0, 1));
     } catch (ExitUtil.ExitException e) {
       fail();
     }
@@ -28,8 +28,8 @@ public class TestTimelineServiceClientIntegration {
 
   @AfterClass
   public static void tearDownClass() throws Exception {
-    if (server != null) {
-      server.stop();
+    if (auxService != null) {
+      auxService.stop();
     }
   }
 

+ 0 - 136
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelServiceManager.java

@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 org.apache.hadoop.yarn.server.timelineservice.aggregator;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-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.service.CompositeService;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-
-/**
- * Class that manages adding and removing app level aggregator services and
- * their lifecycle. It provides thread safety access to the app level services.
- *
- * It is a singleton, and instances should be obtained via
- * {@link #getInstance()}.
- */
-@Private
-@Unstable
-public class AppLevelServiceManager extends CompositeService {
-  private static final Log LOG =
-      LogFactory.getLog(AppLevelServiceManager.class);
-  private static final AppLevelServiceManager INSTANCE =
-      new AppLevelServiceManager();
-
-  // access to this map is synchronized with the map itself
-  private final Map<String,AppLevelAggregatorService> services =
-      Collections.synchronizedMap(
-          new HashMap<String,AppLevelAggregatorService>());
-
-  static AppLevelServiceManager getInstance() {
-    return INSTANCE;
-  }
-
-  AppLevelServiceManager() {
-    super(AppLevelServiceManager.class.getName());
-  }
-
-  /**
-   * 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.
-   *
-   * @throws YarnRuntimeException if there was any exception in initializing and
-   * starting the app level service
-   * @return whether it was added successfully
-   */
-  public boolean addService(String appId) {
-    synchronized (services) {
-      AppLevelAggregatorService service = services.get(appId);
-      if (service == null) {
-        try {
-          service = new AppLevelAggregatorService(appId);
-          // initialize, start, and add it to the parent service so it can be
-          // cleaned up when the parent shuts down
-          service.init(getConfig());
-          service.start();
-          services.put(appId, service);
-          LOG.info("the application aggregator service for " + appId +
-              " was added");
-          return true;
-        } catch (Exception e) {
-          throw new YarnRuntimeException(e);
-        }
-      } else {
-        String msg = "the application aggregator service for " + appId +
-            " already exists!";
-        LOG.error(msg);
-        return false;
-      }
-    }
-  }
-
-  /**
-   * 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
-   * change is made.
-   *
-   * @return whether it was removed successfully
-   */
-  public boolean removeService(String appId) {
-    synchronized (services) {
-      AppLevelAggregatorService service = services.remove(appId);
-      if (service == null) {
-        String msg = "the application aggregator service for " + appId +
-            " does not exist!";
-        LOG.error(msg);
-        return false;
-      } else {
-        // stop the service to do clean up
-        service.stop();
-        LOG.info("the application aggregator service for " + appId +
-            " was removed");
-        return true;
-      }
-    }
-  }
-
-  /**
-   * Returns the app level aggregator service for the specified application id.
-   *
-   * @return the app level aggregator service or null if it does not exist
-   */
-  public AppLevelAggregatorService getService(String appId) {
-    return services.get(appId);
-  }
-
-  /**
-   * Returns whether the app level aggregator service for the specified
-   * application id exists.
-   */
-  public boolean hasService(String appId) {
-    return services.containsKey(appId);
-  }
-}

+ 0 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelServiceManagerProvider.java

@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 org.apache.hadoop.yarn.server.timelineservice.aggregator;
-
-import com.google.inject.Provider;
-
-/**
- * A guice provider that provides a global singleton instance of
- * AppLevelServiceManager.
- */
-public class AppLevelServiceManagerProvider
-    implements Provider<AppLevelServiceManager> {
-  @Override
-  public AppLevelServiceManager get() {
-    return AppLevelServiceManager.getInstance();
-  }
-}

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelAggregatorService.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/AppLevelTimelineAggregator.java

@@ -30,12 +30,12 @@ import org.apache.hadoop.conf.Configuration;
  */
 @Private
 @Unstable
-public class AppLevelAggregatorService extends BaseAggregatorService {
+public class AppLevelTimelineAggregator extends TimelineAggregator {
   private final String applicationId;
   // TODO define key metadata such as flow metadata, user, and queue
 
-  public AppLevelAggregatorService(String applicationId) {
-    super(AppLevelAggregatorService.class.getName() + " - " + applicationId);
+  public AppLevelTimelineAggregator(String applicationId) {
+    super(AppLevelTimelineAggregator.class.getName() + " - " + applicationId);
     this.applicationId = applicationId;
   }
 

+ 40 - 96
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeAggregatorServer.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java

@@ -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();
     }
   }
 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/BaseAggregatorService.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregator.java

@@ -31,15 +31,15 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
  * Service that handles writes to the timeline service and writes them to the
  * backing storage.
  *
- * Classes that extend this can add their own lifecycle management or
+ * Classes that extend this can putIfAbsent their own lifecycle management or
  * customization of request handling.
  */
 @Private
 @Unstable
-public class BaseAggregatorService extends CompositeService {
-  private static final Log LOG = LogFactory.getLog(BaseAggregatorService.class);
+public abstract class TimelineAggregator extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(TimelineAggregator.class);
 
-  public BaseAggregatorService(String name) {
+  public TimelineAggregator(String name) {
     super(name);
   }
 

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeAggregatorWebService.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorWebService.java

@@ -53,9 +53,9 @@ import com.google.inject.Singleton;
 @Unstable
 @Singleton
 @Path("/ws/v2/timeline")
-public class PerNodeAggregatorWebService {
+public class TimelineAggregatorWebService {
   private static final Log LOG =
-      LogFactory.getLog(PerNodeAggregatorWebService.class);
+      LogFactory.getLog(TimelineAggregatorWebService.class);
 
   private @Context ServletContext context;
 
@@ -128,7 +128,7 @@ public class PerNodeAggregatorWebService {
       if (appId == null) {
         return Response.status(Response.Status.BAD_REQUEST).build();
       }
-      AppLevelAggregatorService service = getAggregatorService(req, appId);
+      TimelineAggregator service = getAggregatorService(req, appId);
       if (service == null) {
         LOG.error("Application not found");
         throw new NotFoundException(); // different exception?
@@ -156,13 +156,13 @@ public class PerNodeAggregatorWebService {
     }
   }
 
-  private AppLevelAggregatorService
+  private TimelineAggregator
       getAggregatorService(HttpServletRequest req, String appIdToParse) {
     String appIdString = parseApplicationId(appIdToParse);
-    final AppLevelServiceManager serviceManager =
-        (AppLevelServiceManager) context.getAttribute(
-            PerNodeAggregatorServer.AGGREGATOR_COLLECTION_ATTR_KEY);
-    return serviceManager.getService(appIdString);
+    final TimelineAggregatorsCollection aggregatorCollection =
+        (TimelineAggregatorsCollection) context.getAttribute(
+            TimelineAggregatorsCollection.AGGREGATOR_COLLECTION_ATTR_KEY);
+    return aggregatorCollection.get(appIdString);
   }
 
   private void init(HttpServletResponse response) {

+ 203 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java

@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.hadoop.yarn.server.timelineservice.aggregator;
+
+import java.net.URI;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+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.HttpServer2;
+import org.apache.hadoop.http.lib.StaticUserWebFilter;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
+
+/**
+ * Class that manages adding and removing aggregators and their lifecycle. It
+ * provides thread safety access to the aggregators inside.
+ *
+ * It is a singleton, and instances should be obtained via
+ * {@link #getInstance()}.
+ */
+@Private
+@Unstable
+public class TimelineAggregatorsCollection extends CompositeService {
+  private static final Log LOG =
+      LogFactory.getLog(TimelineAggregatorsCollection.class);
+  private static final TimelineAggregatorsCollection INSTANCE =
+      new TimelineAggregatorsCollection();
+
+  // access to this map is synchronized with the map itself
+  private final Map<String, TimelineAggregator> aggregators =
+      Collections.synchronizedMap(
+          new HashMap<String, TimelineAggregator>());
+
+  // REST server for this aggregator collection
+  private HttpServer2 timelineRestServer;
+
+  static final String AGGREGATOR_COLLECTION_ATTR_KEY = "aggregator.collection";
+
+  static TimelineAggregatorsCollection getInstance() {
+    return INSTANCE;
+  }
+
+  TimelineAggregatorsCollection() {
+    super(TimelineAggregatorsCollection.class.getName());
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    startWebApp();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (timelineRestServer != null) {
+      timelineRestServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  /**
+   * Put the aggregator into the collection if an aggregator mapped by id does
+   * not exist.
+   *
+   * @throws YarnRuntimeException if there was any exception in initializing and
+   * starting the app level service
+   * @return the aggregator associated with id after the potential put.
+   */
+  public TimelineAggregator putIfAbsent(String id, TimelineAggregator aggregator) {
+    synchronized (aggregators) {
+      TimelineAggregator aggregatorInTable = aggregators.get(id);
+      if (aggregatorInTable == null) {
+        try {
+          // initialize, start, and add it to the collection so it can be
+          // cleaned up when the parent shuts down
+          aggregator.init(getConfig());
+          aggregator.start();
+          aggregators.put(id, aggregator);
+          LOG.info("the aggregator for " + id + " was added");
+          return aggregator;
+        } catch (Exception e) {
+          throw new YarnRuntimeException(e);
+        }
+      } else {
+        String msg = "the aggregator for " + id + " already exists!";
+        LOG.error(msg);
+        return aggregatorInTable;
+      }
+    }
+  }
+
+  /**
+   * Removes the aggregator for the specified 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 remove(String id) {
+    synchronized (aggregators) {
+      TimelineAggregator aggregator = aggregators.remove(id);
+      if (aggregator == null) {
+        String msg = "the aggregator for " + id + " does not exist!";
+        LOG.error(msg);
+        return false;
+      } else {
+        // stop the service to do clean up
+        aggregator.stop();
+        LOG.info("the aggregator service for " + id + " was removed");
+        return true;
+      }
+    }
+  }
+
+  /**
+   * Returns the aggregator for the specified id.
+   *
+   * @return the aggregator or null if it does not exist
+   */
+  public TimelineAggregator get(String id) {
+    return aggregators.get(id);
+  }
+
+  /**
+   * Returns whether the aggregator for the specified id exists in this
+   * collection.
+   */
+  public boolean containsKey(String id) {
+    return aggregators.containsKey(id);
+  }
+
+  /**
+   * Launch the REST web server for this aggregator collection
+   */
+  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(
+          TimelineAggregatorWebService.class.getPackage().getName() + ";"
+              + GenericExceptionHandler.class.getPackage().getName() + ";"
+              + YarnJacksonJaxbJsonProvider.class.getPackage().getName(),
+          "/*");
+      timelineRestServer.setAttribute(AGGREGATOR_COLLECTION_ATTR_KEY,
+          TimelineAggregatorsCollection.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);
+    }
+  }
+}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestAppLevelAggregatorService.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestAppLevelTimelineAggregator.java

@@ -19,5 +19,5 @@
 package org.apache.hadoop.yarn.server.timelineservice.aggregator;
 
 
-public class TestAppLevelAggregatorService {
+public class TestAppLevelTimelineAggregator {
 }

+ 40 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeAggregatorServer.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestPerNodeTimelineAggregatorsAuxService.java

@@ -36,10 +36,10 @@ import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
 import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.junit.Test;
 
-public class TestPerNodeAggregatorServer {
+public class TestPerNodeTimelineAggregatorsAuxService {
   private ApplicationAttemptId appAttemptId;
 
-  public TestPerNodeAggregatorServer() {
+  public TestPerNodeTimelineAggregatorsAuxService() {
     ApplicationId appId =
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
     appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
@@ -47,96 +47,97 @@ public class TestPerNodeAggregatorServer {
 
   @Test
   public void testAddApplication() throws Exception {
-    PerNodeAggregatorServer aggregator = createAggregatorAndAddApplication();
-    // aggregator should have a single app
-    assertTrue(aggregator.hasApplication(
+    PerNodeTimelineAggregatorsAuxService auxService = createAggregatorAndAddApplication();
+    // auxService should have a single app
+    assertTrue(auxService.hasApplication(
         appAttemptId.getApplicationId().toString()));
-    aggregator.close();
+    auxService.close();
   }
 
   @Test
   public void testAddApplicationNonAMContainer() throws Exception {
-    PerNodeAggregatorServer aggregator = createAggregator();
+    PerNodeTimelineAggregatorsAuxService auxService = createAggregator();
 
     ContainerId containerId = getContainerId(2L); // not an AM
     ContainerInitializationContext context =
         mock(ContainerInitializationContext.class);
     when(context.getContainerId()).thenReturn(containerId);
-    aggregator.initializeContainer(context);
-    // aggregator should not have that app
-    assertFalse(aggregator.hasApplication(
+    auxService.initializeContainer(context);
+    // auxService should not have that app
+    assertFalse(auxService.hasApplication(
         appAttemptId.getApplicationId().toString()));
   }
 
   @Test
   public void testRemoveApplication() throws Exception {
-    PerNodeAggregatorServer aggregator = createAggregatorAndAddApplication();
-    // aggregator should have a single app
+    PerNodeTimelineAggregatorsAuxService auxService = createAggregatorAndAddApplication();
+    // auxService should have a single app
     String appIdStr = appAttemptId.getApplicationId().toString();
-    assertTrue(aggregator.hasApplication(appIdStr));
+    assertTrue(auxService.hasApplication(appIdStr));
 
     ContainerId containerId = getAMContainerId();
     ContainerTerminationContext context =
         mock(ContainerTerminationContext.class);
     when(context.getContainerId()).thenReturn(containerId);
-    aggregator.stopContainer(context);
-    // aggregator should not have that app
-    assertFalse(aggregator.hasApplication(appIdStr));
-    aggregator.close();
+    auxService.stopContainer(context);
+    // auxService should not have that app
+    assertFalse(auxService.hasApplication(appIdStr));
+    auxService.close();
   }
 
   @Test
   public void testRemoveApplicationNonAMContainer() throws Exception {
-    PerNodeAggregatorServer aggregator = createAggregatorAndAddApplication();
-    // aggregator should have a single app
+    PerNodeTimelineAggregatorsAuxService auxService = createAggregatorAndAddApplication();
+    // auxService should have a single app
     String appIdStr = appAttemptId.getApplicationId().toString();
-    assertTrue(aggregator.hasApplication(appIdStr));
+    assertTrue(auxService.hasApplication(appIdStr));
 
     ContainerId containerId = getContainerId(2L); // not an AM
     ContainerTerminationContext context =
         mock(ContainerTerminationContext.class);
     when(context.getContainerId()).thenReturn(containerId);
-    aggregator.stopContainer(context);
-    // aggregator should still have that app
-    assertTrue(aggregator.hasApplication(appIdStr));
-    aggregator.close();
+    auxService.stopContainer(context);
+    // auxService should still have that app
+    assertTrue(auxService.hasApplication(appIdStr));
+    auxService.close();
   }
 
   @Test(timeout = 60000)
   public void testLaunch() throws Exception {
     ExitUtil.disableSystemExit();
-    PerNodeAggregatorServer server = null;
+    PerNodeTimelineAggregatorsAuxService auxService = null;
     try {
-      server =
-          PerNodeAggregatorServer.launchServer(new String[0]);
+      auxService =
+          PerNodeTimelineAggregatorsAuxService.launchServer(new String[0]);
     } catch (ExitUtil.ExitException e) {
       assertEquals(0, e.status);
       ExitUtil.resetFirstExitException();
       fail();
     } finally {
-      if (server != null) {
-        server.stop();
+      if (auxService != null) {
+        auxService.stop();
       }
     }
   }
 
-  private PerNodeAggregatorServer createAggregatorAndAddApplication() {
-    PerNodeAggregatorServer aggregator = createAggregator();
+  private PerNodeTimelineAggregatorsAuxService createAggregatorAndAddApplication() {
+    PerNodeTimelineAggregatorsAuxService auxService = createAggregator();
     // create an AM container
     ContainerId containerId = getAMContainerId();
     ContainerInitializationContext context =
         mock(ContainerInitializationContext.class);
     when(context.getContainerId()).thenReturn(containerId);
-    aggregator.initializeContainer(context);
-    return aggregator;
+    auxService.initializeContainer(context);
+    return auxService;
   }
 
-  private PerNodeAggregatorServer createAggregator() {
-    AppLevelServiceManager serviceManager = spy(new AppLevelServiceManager());
-    doReturn(new Configuration()).when(serviceManager).getConfig();
-    PerNodeAggregatorServer aggregator =
-        spy(new PerNodeAggregatorServer(serviceManager));
-    return aggregator;
+  private PerNodeTimelineAggregatorsAuxService createAggregator() {
+    TimelineAggregatorsCollection
+        aggregatorsCollection = spy(new TimelineAggregatorsCollection());
+    doReturn(new Configuration()).when(aggregatorsCollection).getConfig();
+    PerNodeTimelineAggregatorsAuxService auxService =
+        spy(new PerNodeTimelineAggregatorsAuxService(aggregatorsCollection));
+    return auxService;
   }
 
   private ContainerId getAMContainerId() {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestBaseAggregatorService.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregator.java

@@ -18,6 +18,6 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.aggregator;
 
-public class TestBaseAggregatorService {
+public class TestTimelineAggregator {
 
 }

+ 18 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestAppLevelServiceManager.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java

@@ -30,16 +30,17 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
+import com.sun.jersey.core.impl.provider.entity.XMLJAXBElementProvider;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.Test;
 
-public class TestAppLevelServiceManager {
+public class TestTimelineAggregatorsCollection {
 
   @Test(timeout=60000)
   public void testMultithreadedAdd() throws Exception {
-    final AppLevelServiceManager serviceManager =
-        spy(new AppLevelServiceManager());
-    doReturn(new Configuration()).when(serviceManager).getConfig();
+    final TimelineAggregatorsCollection aggregatorCollection =
+        spy(new TimelineAggregatorsCollection());
+    doReturn(new Configuration()).when(aggregatorCollection).getConfig();
 
     final int NUM_APPS = 5;
     List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
@@ -47,7 +48,9 @@ public class TestAppLevelServiceManager {
       final String appId = String.valueOf(i);
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
-          return serviceManager.addService(appId);
+          AppLevelTimelineAggregator aggregator =
+              new AppLevelTimelineAggregator(appId);
+          return (aggregatorCollection.putIfAbsent(appId, aggregator) == aggregator);
         }
       };
       tasks.add(task);
@@ -63,15 +66,15 @@ public class TestAppLevelServiceManager {
     }
     // check the keys
     for (int i = 0; i < NUM_APPS; i++) {
-      assertTrue(serviceManager.hasService(String.valueOf(i)));
+      assertTrue(aggregatorCollection.containsKey(String.valueOf(i)));
     }
   }
 
   @Test
   public void testMultithreadedAddAndRemove() throws Exception {
-    final AppLevelServiceManager serviceManager =
-        spy(new AppLevelServiceManager());
-    doReturn(new Configuration()).when(serviceManager).getConfig();
+    final TimelineAggregatorsCollection aggregatorCollection =
+        spy(new TimelineAggregatorsCollection());
+    doReturn(new Configuration()).when(aggregatorCollection).getConfig();
 
     final int NUM_APPS = 5;
     List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
@@ -79,8 +82,11 @@ public class TestAppLevelServiceManager {
       final String appId = String.valueOf(i);
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
-          return serviceManager.addService(appId) &&
-              serviceManager.removeService(appId);
+          AppLevelTimelineAggregator aggregator =
+              new AppLevelTimelineAggregator(appId);
+          boolean successPut =
+              (aggregatorCollection.putIfAbsent(appId, aggregator) == aggregator);
+          return successPut && aggregatorCollection.remove(appId);
         }
       };
       tasks.add(task);
@@ -96,7 +102,7 @@ public class TestAppLevelServiceManager {
     }
     // check the keys
     for (int i = 0; i < NUM_APPS; i++) {
-      assertFalse(serviceManager.hasService(String.valueOf(i)));
+      assertFalse(aggregatorCollection.containsKey(String.valueOf(i)));
     }
   }
 }