Prechádzať zdrojové kódy

Merge r1569890 through r1571508 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1571509 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 rokov pred
rodič
commit
0809f32f4b
58 zmenil súbory, kde vykonal 1872 pridanie a 1040 odobranie
  1. 2 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyUsers.java
  3. 90 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java
  4. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  5. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  6. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
  7. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  8. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
  9. 277 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  10. 120 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
  11. 18 0
      hadoop-yarn-project/CHANGES.txt
  12. 10 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java
  13. 48 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java
  14. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java
  15. 24 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java
  16. 35 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java
  17. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java
  18. 70 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  19. 10 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
  20. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSClientImpl.java
  21. 13 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  22. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
  23. 23 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
  24. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
  25. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/package-info.java
  26. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
  27. 47 57
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  28. 43 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timeline/TestTimelineRecords.java
  29. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
  30. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
  31. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  32. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
  33. 0 131
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineReader.java
  34. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/EntityIdentifier.java
  35. 24 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/GenericObjectMapper.java
  36. 144 125
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/LeveldbTimelineStore.java
  37. 54 44
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/MemoryTimelineStore.java
  38. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/NameValuePair.java
  39. 141 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineReader.java
  40. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineStore.java
  41. 12 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineWriter.java
  42. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/package-info.java
  43. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
  44. 25 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TimelineWebServices.java
  45. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
  46. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerImpl.java
  47. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
  48. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TestGenericObjectMapper.java
  49. 6 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TestLeveldbTimelineStore.java
  50. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TestMemoryTimelineStore.java
  51. 204 177
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineStoreTestUtils.java
  52. 44 44
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestTimelineWebServices.java
  53. 10 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  54. 66 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  55. 23 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java
  56. 42 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java
  57. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  58. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -124,6 +124,8 @@ Trunk (Unreleased)
     HADOOP-10342. Add a new method to UGI to use a Kerberos login subject to
     build a new UGI. (Larry McCay via omalley)
 
+    HADOOP-9968. Makes ProxyUsers to work with NetGroups (Benoy Antony via ddas)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyUsers.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.security.authorize;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.Map.Entry;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
@@ -68,8 +70,11 @@ public class ProxyUsers {
     String regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_GROUPS;
     Map<String,String> allMatchKeys = conf.getValByRegex(regex);
     for(Entry<String, String> entry : allMatchKeys.entrySet()) {
-      proxyGroups.put(entry.getKey(), 
-          StringUtils.getStringCollection(entry.getValue()));
+      Collection<String> groups = StringUtils.getStringCollection(entry.getValue());
+      proxyGroups.put(entry.getKey(), groups );
+      //cache the groups. This is needed for NetGroups
+      Groups.getUserToGroupsMappingService(conf).cacheGroupsAdd(
+          new ArrayList<String>(groups));
     }
 
     // now hosts

+ 90 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestProxyUsers.java

@@ -17,23 +17,113 @@
  */
 package org.apache.hadoop.security.authorize;
 
+import java.io.IOException;
 import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import org.junit.Test;
 import static org.junit.Assert.*;
 
+
 public class TestProxyUsers {
+  private static final Log LOG =
+    LogFactory.getLog(TestProxyUsers.class);
   private static final String REAL_USER_NAME = "proxier";
   private static final String PROXY_USER_NAME = "proxied_user";
   private static final String[] GROUP_NAMES =
     new String[] { "foo_group" };
+  private static final String[] NETGROUP_NAMES =
+    new String[] { "@foo_group" };
   private static final String[] OTHER_GROUP_NAMES =
     new String[] { "bar_group" };
   private static final String PROXY_IP = "1.2.3.4";
 
+  /**
+   * Test the netgroups (groups in ACL rules that start with @)
+   *
+   * This is a  manual test because it requires:
+   *   - host setup
+   *   - native code compiled
+   *   - specify the group mapping class
+   *
+   * Host setup:
+   *
+   * /etc/nsswitch.conf should have a line like this:
+   * netgroup: files
+   *
+   * /etc/netgroup should be (the whole file):
+   * foo_group (,proxied_user,)
+   *
+   * To run this test:
+   *
+   * export JAVA_HOME='path/to/java'
+   * mvn test \
+   *   -Dtest=TestProxyUsers \
+   *   -DTestProxyUsersGroupMapping=$className \
+   *   
+   * where $className is one of the classes that provide group
+   * mapping services, i.e. classes that implement
+   * GroupMappingServiceProvider interface, at this time:
+   *   - org.apache.hadoop.security.JniBasedUnixGroupsNetgroupMapping
+   *   - org.apache.hadoop.security.ShellBasedUnixGroupsNetgroupMapping
+   *
+   */
+  
+  @Test
+  public void testNetgroups () throws IOException{
+  
+    if(!NativeCodeLoader.isNativeCodeLoaded()) {
+      LOG.info("Not testing netgroups, " +
+        "this test only runs when native code is compiled");
+      return;
+    }
+
+    String groupMappingClassName =
+      System.getProperty("TestProxyUsersGroupMapping");
+
+    if(groupMappingClassName == null) {
+      LOG.info("Not testing netgroups, no group mapping class specified, " +
+        "use -DTestProxyUsersGroupMapping=$className to specify " +
+        "group mapping class (must implement GroupMappingServiceProvider " +
+        "interface and support netgroups)");
+      return;
+    }
+
+    LOG.info("Testing netgroups using: " + groupMappingClassName);
+
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_GROUP_MAPPING,
+      groupMappingClassName);
+
+    conf.set(
+        ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
+        StringUtils.join(",", Arrays.asList(NETGROUP_NAMES)));
+    conf.set(
+        ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
+        PROXY_IP);
+    
+    ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+    Groups groups = Groups.getUserToGroupsMappingService(conf);
+
+    // try proxying a group that's allowed
+    UserGroupInformation realUserUgi = UserGroupInformation
+    .createRemoteUser(REAL_USER_NAME);
+
+    UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
+        PROXY_USER_NAME, realUserUgi, groups.getGroups(PROXY_USER_NAME).toArray(
+            new String[groups.getGroups(PROXY_USER_NAME).size()]));
+
+    assertAuthorized(proxyUserUgi, PROXY_IP);
+  }
+
   @Test
   public void testProxyUsers() throws Exception {
     Configuration conf = new Configuration();

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -353,6 +353,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5698. Use protobuf to serialize / deserialize FSImage. (See breakdown
     of tasks below for features and contributors)
 
+    HDFS-5776 Support 'hedged' reads in DFSClient (Liang Xie via stack)
+
   IMPROVEMENTS
 
     HDFS-5781. Use an array to record the mapping between FSEditLogOpCode and 

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -80,6 +80,10 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.net.SocketFactory;
 
@@ -175,6 +179,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum.Type;
 import org.apache.hadoop.util.Progressable;
@@ -224,6 +229,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   private final CachingStrategy defaultReadCachingStrategy;
   private final CachingStrategy defaultWriteCachingStrategy;
   private final ClientContext clientContext;
+  private volatile long hedgedReadThresholdMillis;
+  private static DFSHedgedReadMetrics HEDGED_READ_METRIC =
+      new DFSHedgedReadMetrics();
+  private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
   
   /**
    * DFSClient configuration 
@@ -576,6 +585,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     this.clientContext = ClientContext.get(
         conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
         dfsClientConf);
+    this.hedgedReadThresholdMillis = conf.getLong(
+        DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
+        DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS);
+    int numThreads = conf.getInt(
+        DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
+        DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE);
+    if (numThreads > 0) {
+      this.initThreadsNumForHedgedReads(numThreads);
+    }
   }
   
   /**
@@ -2720,4 +2738,64 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
       }
     }
   }
+
+  /**
+   * Create hedged reads thread pool, HEDGED_READ_THREAD_POOL, if
+   * it does not already exist.
+   * @param num Number of threads for hedged reads thread pool.
+   * If zero, skip hedged reads thread pool creation.
+   */
+  private synchronized void initThreadsNumForHedgedReads(int num) {
+    if (num <= 0 || HEDGED_READ_THREAD_POOL != null) return;
+    HEDGED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
+        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+        new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIndex =
+            new AtomicInteger(0); 
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("hedgedRead-" +
+              threadIndex.getAndIncrement());
+            return t;
+          }
+        },
+        new ThreadPoolExecutor.CallerRunsPolicy() {
+
+      @Override
+      public void rejectedExecution(Runnable runnable,
+          ThreadPoolExecutor e) {
+        LOG.info("Execution rejected, Executing in current thread");
+        HEDGED_READ_METRIC.incHedgedReadOpsInCurThread();
+        // will run in the current thread
+        super.rejectedExecution(runnable, e);
+      }
+    });
+    HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using hedged reads; pool threads=" + num);
+    }
+  }
+
+  long getHedgedReadTimeout() {
+    return this.hedgedReadThresholdMillis;
+  }
+
+  @VisibleForTesting
+  void setHedgedReadTimeout(long timeoutMillis) {
+    this.hedgedReadThresholdMillis = timeoutMillis;
+  }
+
+  ThreadPoolExecutor getHedgedReadsThreadPool() {
+    return HEDGED_READ_THREAD_POOL;
+  }
+
+  boolean isHedgedReadsEnabled() {
+    return (HEDGED_READ_THREAD_POOL != null) &&
+      HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;
+  }
+
+  DFSHedgedReadMetrics getHedgedReadMetrics() {
+    return HEDGED_READ_METRIC;
+  }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java

@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs;
-import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -46,4 +45,6 @@ public class DFSClientFaultInjector {
   public boolean failPacket() {
     return false;
   }
+
+  public void startFetchFromDatanode() {}
 }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -595,4 +595,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
   public static final String  DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.http.client.failover.sleep.max.millis";
   public static final int     DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
+
+  // hedged read properties
+  public static final String DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS =
+      "dfs.client.hedged.read.threshold.millis";
+  public static final long DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS =
+      500;
+
+  public static final String DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE =
+      "dfs.client.hedged.read.threadpool.size";
+  public static final int DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE = 0;
 }

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java

@@ -0,0 +1,55 @@
+/**
+ * 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.hdfs;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * The client-side metrics for hedged read feature.
+ * This class has a number of metrics variables that are publicly accessible,
+ * we can grab them from client side, like HBase.
+ */
+public class DFSHedgedReadMetrics {
+  public AtomicLong hedgedReadOps = new AtomicLong();
+  public AtomicLong hedgedReadOpsWin = new AtomicLong();
+  public AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
+
+  public void incHedgedReadOps() {
+    hedgedReadOps.incrementAndGet();
+  }
+
+  public void incHedgedReadOpsInCurThread() {
+    hedgedReadOpsInCurThread.incrementAndGet();
+  }
+
+  public void incHedgedReadWins() {
+    hedgedReadOpsWin.incrementAndGet();
+  }
+
+  public long getHedgedReadOps() {
+    return hedgedReadOps.longValue();
+  }
+
+  public long getHedgedReadOpsInCurThread() {
+    return hedgedReadOpsInCurThread.longValue();
+  }
+
+  public long getHedgedReadWins() {
+    return hedgedReadOpsWin.longValue();
+  }
+}

+ 277 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -17,13 +17,12 @@
  */
 package org.apache.hadoop.hdfs;
 
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.util.AbstractMap;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -32,9 +31,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ByteBufferReadable;
 import org.apache.hadoop.fs.ByteBufferUtil;
@@ -54,15 +58,12 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.unix.DomainSocket;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.IdentityHashStore;
@@ -555,7 +556,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       assert (target==pos) : "Wrong postion " + pos + " expect " + target;
       long offsetIntoBlock = target - targetBlock.getStartOffset();
 
-      DNAddrPair retval = chooseDataNode(targetBlock);
+      DNAddrPair retval = chooseDataNode(targetBlock, null);
       chosenNode = retval.info;
       InetSocketAddress targetAddr = retval.addr;
 
@@ -863,32 +864,30 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       corruptedBlockMap.put(blk, dnSet);
     }
   }
-      
-  private DNAddrPair chooseDataNode(LocatedBlock block)
-    throws IOException {
+
+  private DNAddrPair chooseDataNode(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes) throws IOException {
     while (true) {
       DatanodeInfo[] nodes = block.getLocations();
       try {
-        DatanodeInfo chosenNode = bestNode(nodes, deadNodes);
-        final String dnAddr =
-            chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
-        }
-        InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
-        return new DNAddrPair(chosenNode, targetAddr);
+        return getBestNodeDNAddrPair(nodes, ignoredNodes);
       } catch (IOException ie) {
+        String errMsg =
+          getBestNodeDNAddrPairErrorString(nodes, deadNodes, ignoredNodes);
         String blockInfo = block.getBlock() + " file=" + src;
         if (failures >= dfsClient.getMaxBlockAcquireFailures()) {
-          throw new BlockMissingException(src, "Could not obtain block: " + blockInfo,
-                                          block.getStartOffset());
+          String description = "Could not obtain block: " + blockInfo;
+          DFSClient.LOG.warn(description + errMsg
+              + ". Throwing a BlockMissingException");
+          throw new BlockMissingException(src, description,
+              block.getStartOffset());
         }
         
         if (nodes == null || nodes.length == 0) {
           DFSClient.LOG.info("No node available for " + blockInfo);
         }
         DFSClient.LOG.info("Could not obtain " + block.getBlock()
-            + " from any node: " + ie
+            + " from any node: " + ie + errMsg
             + ". Will get new block locations from namenode and retry...");
         try {
           // Introducing a random factor to the wait time before another retry.
@@ -914,21 +913,99 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         continue;
       }
     }
-  } 
-      
+  }
+
+  /**
+   * Get the best node.
+   * @param nodes Nodes to choose from.
+   * @param ignoredNodes Do not chose nodes in this array (may be null)
+   * @return The DNAddrPair of the best node.
+   * @throws IOException
+   */
+  private DNAddrPair getBestNodeDNAddrPair(final DatanodeInfo[] nodes,
+      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    DatanodeInfo chosenNode = bestNode(nodes, deadNodes, ignoredNodes);
+    final String dnAddr =
+        chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+    }
+    InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
+    return new DNAddrPair(chosenNode, targetAddr);
+  }
+
+  private static String getBestNodeDNAddrPairErrorString(
+      DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
+      DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
+    StringBuilder errMsgr = new StringBuilder(
+        " No live nodes contain current block ");
+    errMsgr.append("Block locations:");
+    for (DatanodeInfo datanode : nodes) {
+      errMsgr.append(" ");
+      errMsgr.append(datanode.toString());
+    }
+    errMsgr.append(" Dead nodes: ");
+    for (DatanodeInfo datanode : deadNodes.keySet()) {
+      errMsgr.append(" ");
+      errMsgr.append(datanode.toString());
+    }
+    if (ignoredNodes != null) {
+      errMsgr.append(" Ignored nodes: ");
+      for (DatanodeInfo datanode : ignoredNodes) {
+        errMsgr.append(" ");
+        errMsgr.append(datanode.toString());
+      }
+    }
+    return errMsgr.toString();
+  }
+
   private void fetchBlockByteRange(LocatedBlock block, long start, long end,
       byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
-    //
-    // Connect to best DataNode for desired Block, with potential offset
-    //
+    block = getBlockAt(block.getStartOffset(), false);
+    while (true) {
+      DNAddrPair addressPair = chooseDataNode(block, null);
+      try {
+        actualGetFromOneDataNode(addressPair, block, start, end, buf, offset,
+            corruptedBlockMap);
+        return;
+      } catch (IOException e) {
+        // Ignore. Already processed inside the function.
+        // Loop through to try the next node.
+      }
+    }
+  }
+
+  private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
+      final LocatedBlock block, final long start, final long end,
+      final ByteBuffer bb,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+      final CountDownLatch latch) {
+    return new Callable<ByteBuffer>() {
+      @Override
+      public ByteBuffer call() throws Exception {
+        byte[] buf = bb.array();
+        int offset = bb.position();
+        actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
+            corruptedBlockMap);
+        latch.countDown();
+        return bb;
+      }
+    };
+  }
+
+  private void actualGetFromOneDataNode(final DNAddrPair datanode,
+      LocatedBlock block, final long start, final long end, byte[] buf,
+      int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    DFSClientFaultInjector.get().startFetchFromDatanode();
     int refetchToken = 1; // only need to get a new access token once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
-    
+
     while (true) {
       // cached block locations may have been updated by chooseDataNode()
-      // or fetchBlockAt(). Always get the latest list of locations at the 
+      // or fetchBlockAt(). Always get the latest list of locations at the
       // start of the loop.
       CachingStrategy curCachingStrategy;
       boolean allowShortCircuitLocalReads;
@@ -937,11 +1014,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         curCachingStrategy = cachingStrategy;
         allowShortCircuitLocalReads = !shortCircuitForbidden();
       }
-      DNAddrPair retval = chooseDataNode(block);
-      DatanodeInfo chosenNode = retval.info;
-      InetSocketAddress targetAddr = retval.addr;
+      DatanodeInfo chosenNode = datanode.info;
+      InetSocketAddress targetAddr = datanode.addr;
       BlockReader reader = null;
-          
+
       try {
         Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
         int len = (int) (end - start + 1);
@@ -969,11 +1045,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         return;
       } catch (ChecksumException e) {
-        DFSClient.LOG.warn("fetchBlockByteRange(). Got a checksum exception for " +
-                 src + " at " + block.getBlock() + ":" + 
-                 e.getPos() + " from " + chosenNode);
+        String msg = "fetchBlockByteRange(). Got a checksum exception for "
+            + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
+            + chosenNode;
+        DFSClient.LOG.warn(msg);
         // we want to remember what we have tried
         addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
+        addToDeadNodes(chosenNode);
+        throw new IOException(msg);
       } catch (IOException e) {
         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
@@ -985,22 +1064,164 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           continue;
         } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
           refetchToken--;
-          fetchBlockAt(block.getStartOffset());
+          try {
+            fetchBlockAt(block.getStartOffset());
+          } catch (IOException fbae) {
+            // ignore IOE, since we can retry it later in a loop
+          }
           continue;
         } else {
-          DFSClient.LOG.warn("Failed to connect to " + targetAddr + 
-              " for file " + src + " for block " + block.getBlock() + ":" + e);
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Connection failure ", e);
-          }
+          String msg = "Failed to connect to " + targetAddr + " for file "
+              + src + " for block " + block.getBlock() + ":" + e;
+          DFSClient.LOG.warn("Connection failure: " + msg, e);
+          addToDeadNodes(chosenNode);
+          throw new IOException(msg);
         }
       } finally {
         if (reader != null) {
           reader.close();
         }
       }
-      // Put chosen node into dead list, continue
-      addToDeadNodes(chosenNode);
+    }
+  }
+
+  /**
+   * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
+   * int, Map)} except we start up a second, parallel, 'hedged' read
+   * if the first read is taking longer than configured amount of
+   * time.  We then wait on which ever read returns first.
+   * 
+   * @param block
+   * @param start
+   * @param end
+   * @param buf
+   * @param offset
+   * @param corruptedBlockMap
+   * @throws IOException
+   */
+  private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
+      long end, byte[] buf, int offset,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    ArrayList<Future<ByteBuffer>> futures = null;
+    ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
+    ByteBuffer bb = null;
+    int len = (int) (end - start + 1);
+    block = getBlockAt(block.getStartOffset(), false);
+    // Latch shared by all outstanding reads.  First to finish closes
+    CountDownLatch hasReceivedResult = new CountDownLatch(1);
+    while (true) {
+      DNAddrPair chosenNode = null;
+      Future<ByteBuffer> future = null;
+      // futures is null if there is no request already executing.
+      if (futures == null) {
+        // chooseDataNode is a commitment.  If no node, we go to
+        // the NN to reget block locations.  Only go here on first read.
+        chosenNode = chooseDataNode(block, ignored);
+        bb = ByteBuffer.wrap(buf, offset, len);
+        future = getHedgedReadFuture(chosenNode, block, start, end, bb,
+          corruptedBlockMap, hasReceivedResult);
+        try {
+          future.get(dfsClient.getHedgedReadTimeout(), TimeUnit.MILLISECONDS);
+          return;
+        } catch (TimeoutException e) {
+          if (DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Waited " + dfsClient.getHedgedReadTimeout() +
+              "ms to read from " + chosenNode.info + "; spawning hedged read");
+          }
+          // Ignore this node on next go around.
+          ignored.add(chosenNode.info);
+          dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+          futures = new ArrayList<Future<ByteBuffer>>();
+          futures.add(future);
+          continue; // no need to refresh block locations
+        } catch (InterruptedException e) {
+          // Ignore
+        } catch (ExecutionException e) {
+          // Ignore already logged in the call.
+        }
+      } else {
+        // We are starting up a 'hedged' read.  We have a read already
+        // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
+        // If no nodes to do hedged reads against, pass.
+        try {
+          chosenNode = getBestNodeDNAddrPair(block.getLocations(), ignored);
+          bb = ByteBuffer.allocate(len);
+          future = getHedgedReadFuture(chosenNode, block, start, end, bb,
+            corruptedBlockMap, hasReceivedResult);
+          futures.add(future);
+        } catch (IOException ioe) {
+          if (DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Failed getting node for hedged read: " +
+              ioe.getMessage());
+          }
+        }
+        // if not succeeded. Submit callables for each datanode in a loop, wait
+        // for a fixed interval and get the result from the fastest one.
+        try {
+          ByteBuffer result = getFirstToComplete(futures, hasReceivedResult);
+          // cancel the rest.
+          cancelAll(futures);
+          if (result.array() != buf) { // compare the array pointers
+            dfsClient.getHedgedReadMetrics().incHedgedReadWins();
+            System.arraycopy(result.array(), result.position(), buf, offset,
+                len);
+          } else {
+            dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+          }
+          return;
+        } catch (InterruptedException ie) {
+          // Ignore
+        } catch (ExecutionException e) {
+          // exception already handled in the call method. getFirstToComplete
+          // will remove the failing future from the list. nothing more to do.
+        }
+        // We got here if exception.  Ignore this node on next go around.
+        ignored.add(chosenNode.info);
+      }
+      // executed if we get an error from a data node
+      block = getBlockAt(block.getStartOffset(), false);
+    }
+  }
+
+  private Future<ByteBuffer> getHedgedReadFuture(final DNAddrPair chosenNode,
+      final LocatedBlock block, long start,
+      final long end, final ByteBuffer bb,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+      final CountDownLatch hasReceivedResult) {
+    Callable<ByteBuffer> getFromDataNodeCallable =
+        getFromOneDataNode(chosenNode, block, start, end, bb,
+          corruptedBlockMap, hasReceivedResult);
+    return dfsClient.getHedgedReadsThreadPool().submit(getFromDataNodeCallable);
+  }
+
+  private ByteBuffer getFirstToComplete(ArrayList<Future<ByteBuffer>> futures,
+      CountDownLatch latch) throws ExecutionException, InterruptedException {
+    latch.await();
+    for (Future<ByteBuffer> future : futures) {
+      if (future.isDone()) {
+        try {
+          return future.get();
+        } catch (ExecutionException e) {
+          // already logged in the Callable
+          futures.remove(future);
+          throw e;
+        }
+      }
+    }
+    throw new InterruptedException("latch has counted down to zero but no"
+        + "result available yet, for safety try to request another one from"
+        + "outside loop, this should be rare");
+  }
+
+  private void cancelAll(List<Future<ByteBuffer>> futures) {
+    for (Future<ByteBuffer> future : futures) {
+      // Unfortunately, hdfs reads do not take kindly to interruption.
+      // Threads return a variety of interrupted-type exceptions but
+      // also complaints about invalid pbs -- likely because read
+      // is interrupted before gets whole pb.  Also verbose WARN
+      // logging.  So, for now, do not interrupt running read.
+      future.cancel(false);
     }
   }
 
@@ -1070,8 +1291,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long targetStart = position - blk.getStartOffset();
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
       try {
-        fetchBlockByteRange(blk, targetStart, 
-            targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
+        if (dfsClient.isHedgedReadsEnabled()) {
+          hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead
+              - 1, buffer, offset, corruptedBlockMap);
+        } else {
+          fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
+              buffer, offset, corruptedBlockMap);
+        }
       } finally {
         // Check and report if any block replicas are corrupted.
         // BlockMissingException may be caught if all block replicas are
@@ -1265,12 +1491,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Pick the best node from which to stream the data.
    * Entries in <i>nodes</i> are already in the priority order
    */
-  static DatanodeInfo bestNode(DatanodeInfo nodes[], 
-                               AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes)
-                               throws IOException {
-    if (nodes != null) { 
+  static DatanodeInfo bestNode(DatanodeInfo nodes[],
+      AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes,
+      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    if (nodes != null) {
       for (int i = 0; i < nodes.length; i++) {
-        if (!deadNodes.containsKey(nodes[i])) {
+        if (!deadNodes.containsKey(nodes[i])
+            && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
           return nodes[i];
         }
       }

+ 120 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java

@@ -20,9 +20,14 @@ package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
@@ -33,6 +38,9 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.log4j.Level;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 /**
  * This class tests the DFS positional read functionality in a single node
@@ -44,9 +52,10 @@ public class TestPread {
   boolean simulatedStorage = false;
 
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
+    int replication = 3;// We need > 1 blocks to test out the hedged reads.
     // test empty file open and read
     DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 0,
-        blockSize, (short) 1, seed);
+      blockSize, (short)replication, seed);
     FSDataInputStream in = fileSys.open(name);
     byte[] buffer = new byte[12 * blockSize];
     in.readFully(0, buffer, 0, 0);
@@ -191,26 +200,128 @@ public class TestPread {
     assertTrue(fileSys.delete(name, true));
     assertTrue(!fileSys.exists(name));
   }
-  
+
+  private Callable<Void> getPReadFileCallable(final FileSystem fileSys,
+      final Path file) {
+    return new Callable<Void>() {
+      public Void call() throws IOException {
+        pReadFile(fileSys, file);
+        return null;
+      }
+    };
+  }
+
   /**
    * Tests positional read in DFS.
    */
   @Test
   public void testPreadDFS() throws IOException {
-    dfsPreadTest(false, true); //normal pread
-    dfsPreadTest(true, true); //trigger read code path without transferTo.
+    Configuration conf = new Configuration();
+    dfsPreadTest(conf, false, true); // normal pread
+    dfsPreadTest(conf, true, true); // trigger read code path without
+                                    // transferTo.
   }
   
   @Test
   public void testPreadDFSNoChecksum() throws IOException {
+    Configuration conf = new Configuration();
     ((Log4JLogger)DataTransferProtocol.LOG).getLogger().setLevel(Level.ALL);
-    dfsPreadTest(false, false);
-    dfsPreadTest(true, false);
+    dfsPreadTest(conf, false, false);
+    dfsPreadTest(conf, true, false);
   }
   
-  private void dfsPreadTest(boolean disableTransferTo, boolean verifyChecksum)
+  /**
+   * Tests positional read in DFS, with hedged reads enabled.
+   */
+  @Test
+  public void testHedgedPreadDFSBasic() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE, 5);
+    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS, 100);
+    dfsPreadTest(conf, false, true); // normal pread
+    dfsPreadTest(conf, true, true); // trigger read code path without
+                                    // transferTo.
+  }
+
+  @Test
+  public void testMaxOutHedgedReadPool() throws IOException,
+      InterruptedException, ExecutionException {
+    Configuration conf = new Configuration();
+    int numHedgedReadPoolThreads = 5;
+    final int initialHedgedReadTimeoutMillis = 500;
+    final int fixedSleepIntervalMillis = 50;
+    conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
+        numHedgedReadPoolThreads);
+    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
+        initialHedgedReadTimeoutMillis);
+
+    // Set up the InjectionHandler
+    DFSClientFaultInjector.instance = Mockito
+        .mock(DFSClientFaultInjector.class);
+    DFSClientFaultInjector injector = DFSClientFaultInjector.instance;
+    // make preads sleep for 50ms
+    Mockito.doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        Thread.sleep(fixedSleepIntervalMillis);
+        return null;
+      }
+    }).when(injector).startFetchFromDatanode();
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .format(true).build();
+    DistributedFileSystem fileSys = cluster.getFileSystem();
+    DFSClient dfsClient = fileSys.getClient();
+    DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
+
+    try {
+      Path file1 = new Path("hedgedReadMaxOut.dat");
+      writeFile(fileSys, file1);
+      // Basic test. Reads complete within timeout. Assert that there were no
+      // hedged reads.
+      pReadFile(fileSys, file1);
+      // assert that there were no hedged reads. 50ms + delta < 500ms
+      assertTrue(metrics.getHedgedReadOps() == 0);
+      assertTrue(metrics.getHedgedReadOpsInCurThread() == 0);
+      /*
+       * Reads take longer than timeout. But, only one thread reading. Assert
+       * that there were hedged reads. But, none of the reads had to run in the
+       * current thread.
+       */
+      dfsClient.setHedgedReadTimeout(50); // 50ms
+      pReadFile(fileSys, file1);
+      // assert that there were hedged reads
+      assertTrue(metrics.getHedgedReadOps() > 0);
+      assertTrue(metrics.getHedgedReadOpsInCurThread() == 0);
+      /*
+       * Multiple threads reading. Reads take longer than timeout. Assert that
+       * there were hedged reads. And that reads had to run in the current
+       * thread.
+       */
+      int factor = 10;
+      int numHedgedReads = numHedgedReadPoolThreads * factor;
+      long initialReadOpsValue = metrics.getHedgedReadOps();
+      ExecutorService executor = Executors.newFixedThreadPool(numHedgedReads);
+      ArrayList<Future<Void>> futures = new ArrayList<Future<Void>>();
+      for (int i = 0; i < numHedgedReads; i++) {
+        futures.add(executor.submit(getPReadFileCallable(fileSys, file1)));
+      }
+      for (int i = 0; i < numHedgedReads; i++) {
+        futures.get(i).get();
+      }
+      assertTrue(metrics.getHedgedReadOps() > initialReadOpsValue);
+      assertTrue(metrics.getHedgedReadOpsInCurThread() > 0);
+      cleanupFile(fileSys, file1);
+      executor.shutdown();
+    } finally {
+      fileSys.close();
+      cluster.shutdown();
+      Mockito.reset(injector);
+    }
+  }
+
+  private void dfsPreadTest(Configuration conf, boolean disableTransferTo, boolean verifyChecksum)
       throws IOException {
-    Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
     conf.setLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 4096);
     if (simulatedStorage) {

+ 18 - 0
hadoop-yarn-project/CHANGES.txt

@@ -28,6 +28,8 @@ Release 2.5.0 - UNRELEASED
     YARN-1736. FS: AppSchedulable.assignContainer's priority argument is 
     redundant. (Naren Koneru via kasha)
 
+    YARN-1678. Fair scheduler gabs incessantly about reservations (Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -218,6 +220,15 @@ Release 2.4.0 - UNRELEASED
     YARN-1470. Add audience annotations to MiniYARNCluster. (Anubhav Dhoot
     via kasha)
 
+    YARN-1732. Changed types of related-entities and primary-filters in the
+    timeline-service to be sets instead of maps. (Billie Rinaldi via vinodkv)
+
+    YARN-1687. Renamed user-facing records for the timeline-service to be simply
+    named after 'timeline' instead of 'apptimeline'. (Zhijie Shen via vinodkv)
+
+    YARN-1749. Updated application-history related configs to reflect the latest
+    reality and to be consistently named. (Zhijie Shen via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -330,6 +341,13 @@ Release 2.4.0 - UNRELEASED
     YARN-1071. Enabled ResourceManager to recover cluster metrics
     numDecommissionedNMs after restarting. (Jian He via zjshen)
 
+    YARN-1742. Fixed javadoc of configuration parameter
+    DEFAULT_NM_MIN_HEALTHY_DISKS_FRACTION. (Akira Ajisaka via vinodkv)
+
+    YARN-1686. Fixed NodeManager to properly handle any errors during
+    re-registration after a RESYNC and thus avoid hanging. (Rohith Sharma via
+    vinodkv)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/apptimeline/ATSEntities.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -30,18 +30,18 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 
 /**
- * The class that hosts a list of application timeline entities.
+ * The class that hosts a list of timeline entities.
  */
 @XmlRootElement(name = "entities")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
 @Unstable
-public class ATSEntities {
+public class TimelineEntities {
 
-  private List<ATSEntity> entities =
-      new ArrayList<ATSEntity>();
+  private List<TimelineEntity> entities =
+      new ArrayList<TimelineEntity>();
 
-  public ATSEntities() {
+  public TimelineEntities() {
 
   }
 
@@ -51,7 +51,7 @@ public class ATSEntities {
    * @return a list of entities
    */
   @XmlElement(name = "entities")
-  public List<ATSEntity> getEntities() {
+  public List<TimelineEntity> getEntities() {
     return entities;
   }
 
@@ -61,7 +61,7 @@ public class ATSEntities {
    * @param entity
    *          a single entity
    */
-  public void addEntity(ATSEntity entity) {
+  public void addEntity(TimelineEntity entity) {
     entities.add(entity);
   }
 
@@ -71,7 +71,7 @@ public class ATSEntities {
    * @param entities
    *          a list of entities
    */
-  public void addEntities(List<ATSEntity> entities) {
+  public void addEntities(List<TimelineEntity> entities) {
     this.entities.addAll(entities);
   }
 
@@ -81,7 +81,7 @@ public class ATSEntities {
    * @param entities
    *          a list of entities
    */
-  public void setEntities(List<ATSEntity> entities) {
+  public void setEntities(List<TimelineEntity> entities) {
     this.entities = entities;
   }
 

+ 48 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/apptimeline/ATSEntity.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java

@@ -16,13 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -34,36 +36,36 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 
 /**
  * <p>
- * The class that contains the the meta information of some conceptual entity of
- * an application and its related events. The entity can be an application, an
- * application attempt, a container or whatever the user-defined object.
+ * The class that contains the the meta information of some conceptual entity
+ * and its related events. The entity can be an application, an application
+ * attempt, a container or whatever the user-defined object.
  * </p>
  * 
  * <p>
  * Primary filters will be used to index the entities in
- * <code>ApplicationTimelineStore</code>, such that users should carefully
- * choose the information they want to store as the primary filters. The
- * remaining can be stored as other information.
+ * <code>TimelineStore</code>, such that users should carefully choose the
+ * information they want to store as the primary filters. The remaining can be
+ * stored as other information.
  * </p>
  */
 @XmlRootElement(name = "entity")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
 @Unstable
-public class ATSEntity implements Comparable<ATSEntity> {
+public class TimelineEntity implements Comparable<TimelineEntity> {
 
   private String entityType;
   private String entityId;
   private Long startTime;
-  private List<ATSEvent> events = new ArrayList<ATSEvent>();
-  private Map<String, List<String>> relatedEntities =
-      new HashMap<String, List<String>>();
-  private Map<String, Object> primaryFilters =
-      new HashMap<String, Object>();
+  private List<TimelineEvent> events = new ArrayList<TimelineEvent>();
+  private Map<String, Set<String>> relatedEntities =
+      new HashMap<String, Set<String>>();
+  private Map<String, Set<Object>> primaryFilters =
+      new HashMap<String, Set<Object>>();
   private Map<String, Object> otherInfo =
       new HashMap<String, Object>();
 
-  public ATSEntity() {
+  public TimelineEntity() {
 
   }
 
@@ -133,7 +135,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @return a list of events related to the entity
    */
   @XmlElement(name = "events")
-  public List<ATSEvent> getEvents() {
+  public List<TimelineEvent> getEvents() {
     return events;
   }
 
@@ -143,7 +145,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @param event
    *          a single event related to the entity
    */
-  public void addEvent(ATSEvent event) {
+  public void addEvent(TimelineEvent event) {
     events.add(event);
   }
 
@@ -153,7 +155,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @param events
    *          a list of events related to the entity
    */
-  public void addEvents(List<ATSEvent> events) {
+  public void addEvents(List<TimelineEvent> events) {
     this.events.addAll(events);
   }
 
@@ -163,7 +165,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @param events
    *          events a list of events related to the entity
    */
-  public void setEvents(List<ATSEvent> events) {
+  public void setEvents(List<TimelineEvent> events) {
     this.events = events;
   }
 
@@ -173,7 +175,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @return the related entities
    */
   @XmlElement(name = "relatedentities")
-  public Map<String, List<String>> getRelatedEntities() {
+  public Map<String, Set<String>> getRelatedEntities() {
     return relatedEntities;
   }
 
@@ -186,9 +188,9 @@ public class ATSEntity implements Comparable<ATSEntity> {
    *          the entity Id
    */
   public void addRelatedEntity(String entityType, String entityId) {
-    List<String> thisRelatedEntity = relatedEntities.get(entityType);
+    Set<String> thisRelatedEntity = relatedEntities.get(entityType);
     if (thisRelatedEntity == null) {
-      thisRelatedEntity = new ArrayList<String>();
+      thisRelatedEntity = new HashSet<String>();
       relatedEntities.put(entityType, thisRelatedEntity);
     }
     thisRelatedEntity.add(entityId);
@@ -200,10 +202,9 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @param relatedEntities
    *          a map of related entities
    */
-  public void addRelatedEntities(Map<String, List<String>> relatedEntities) {
-    for (Entry<String, List<String>> relatedEntity :
-        relatedEntities.entrySet()) {
-      List<String> thisRelatedEntity =
+  public void addRelatedEntities(Map<String, Set<String>> relatedEntities) {
+    for (Entry<String, Set<String>> relatedEntity : relatedEntities.entrySet()) {
+      Set<String> thisRelatedEntity =
           this.relatedEntities.get(relatedEntity.getKey());
       if (thisRelatedEntity == null) {
         this.relatedEntities.put(
@@ -221,7 +222,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    *          a map of related entities
    */
   public void setRelatedEntities(
-      Map<String, List<String>> relatedEntities) {
+      Map<String, Set<String>> relatedEntities) {
     this.relatedEntities = relatedEntities;
   }
 
@@ -231,7 +232,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @return the primary filters
    */
   @XmlElement(name = "primaryfilters")
-  public Map<String, Object> getPrimaryFilters() {
+  public Map<String, Set<Object>> getPrimaryFilters() {
     return primaryFilters;
   }
 
@@ -244,7 +245,12 @@ public class ATSEntity implements Comparable<ATSEntity> {
    *          the primary filter value
    */
   public void addPrimaryFilter(String key, Object value) {
-    primaryFilters.put(key, value);
+    Set<Object> thisPrimaryFilter = primaryFilters.get(key);
+    if (thisPrimaryFilter == null) {
+      thisPrimaryFilter = new HashSet<Object>();
+      primaryFilters.put(key, thisPrimaryFilter);
+    }
+    thisPrimaryFilter.add(value);
   }
 
   /**
@@ -253,8 +259,17 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @param primaryFilters
    *          a map of primary filters
    */
-  public void addPrimaryFilters(Map<String, Object> primaryFilters) {
-    this.primaryFilters.putAll(primaryFilters);
+  public void addPrimaryFilters(Map<String, Set<Object>> primaryFilters) {
+    for (Entry<String, Set<Object>> primaryFilter : primaryFilters.entrySet()) {
+      Set<Object> thisPrimaryFilter =
+          this.primaryFilters.get(primaryFilter.getKey());
+      if (thisPrimaryFilter == null) {
+        this.primaryFilters.put(
+            primaryFilter.getKey(), primaryFilter.getValue());
+      } else {
+        thisPrimaryFilter.addAll(primaryFilter.getValue());
+      }
+    }
   }
 
   /**
@@ -263,7 +278,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
    * @param primaryFilters
    *          a map of primary filters
    */
-  public void setPrimaryFilters(Map<String, Object> primaryFilters) {
+  public void setPrimaryFilters(Map<String, Set<Object>> primaryFilters) {
     this.primaryFilters = primaryFilters;
   }
 
@@ -339,7 +354,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
       return false;
     if (getClass() != obj.getClass())
       return false;
-    ATSEntity other = (ATSEntity) obj;
+    TimelineEntity other = (TimelineEntity) obj;
     if (entityId == null) {
       if (other.entityId != null)
         return false;
@@ -379,7 +394,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
   }
 
   @Override
-  public int compareTo(ATSEntity other) {
+  public int compareTo(TimelineEntity other) {
     int comparison = entityType.compareTo(other.entityType);
     if (comparison == 0) {
       long thisStartTime =

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/apptimeline/ATSEvent.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -39,13 +39,13 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
 @Unstable
-public class ATSEvent implements Comparable<ATSEvent> {
+public class TimelineEvent implements Comparable<TimelineEvent> {
 
   private long timestamp;
   private String eventType;
   private Map<String, Object> eventInfo = new HashMap<String, Object>();
 
-  public ATSEvent() {
+  public TimelineEvent() {
   }
 
   /**
@@ -132,7 +132,7 @@ public class ATSEvent implements Comparable<ATSEvent> {
   }
 
   @Override
-  public int compareTo(ATSEvent other) {
+  public int compareTo(TimelineEvent other) {
     if (timestamp > other.timestamp) {
       return -1;
     } else if (timestamp < other.timestamp) {
@@ -149,14 +149,14 @@ public class ATSEvent implements Comparable<ATSEvent> {
     if (o == null || getClass() != o.getClass())
       return false;
 
-    ATSEvent atsEvent = (ATSEvent) o;
+    TimelineEvent event = (TimelineEvent) o;
 
-    if (timestamp != atsEvent.timestamp)
+    if (timestamp != event.timestamp)
       return false;
-    if (!eventType.equals(atsEvent.eventType))
+    if (!eventType.equals(event.eventType))
       return false;
-    if (eventInfo != null ? !eventInfo.equals(atsEvent.eventInfo) :
-        atsEvent.eventInfo != null)
+    if (eventInfo != null ? !eventInfo.equals(event.eventInfo) :
+        event.eventInfo != null)
       return false;
 
     return true;

+ 24 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/apptimeline/ATSEvents.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -37,52 +37,52 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
 @Unstable
-public class ATSEvents {
+public class TimelineEvents {
 
-  private List<ATSEventsOfOneEntity> allEvents =
-      new ArrayList<ATSEventsOfOneEntity>();
+  private List<EventsOfOneEntity> allEvents =
+      new ArrayList<EventsOfOneEntity>();
 
-  public ATSEvents() {
+  public TimelineEvents() {
 
   }
 
   /**
-   * Get a list of {@link ATSEventsOfOneEntity} instances
+   * Get a list of {@link EventsOfOneEntity} instances
    * 
-   * @return a list of {@link ATSEventsOfOneEntity} instances
+   * @return a list of {@link EventsOfOneEntity} instances
    */
   @XmlElement(name = "events")
-  public List<ATSEventsOfOneEntity> getAllEvents() {
+  public List<EventsOfOneEntity> getAllEvents() {
     return allEvents;
   }
 
   /**
-   * Add a single {@link ATSEventsOfOneEntity} instance into the existing list
+   * Add a single {@link EventsOfOneEntity} instance into the existing list
    * 
    * @param eventsOfOneEntity
-   *          a single {@link ATSEventsOfOneEntity} instance
+   *          a single {@link EventsOfOneEntity} instance
    */
-  public void addEvent(ATSEventsOfOneEntity eventsOfOneEntity) {
+  public void addEvent(EventsOfOneEntity eventsOfOneEntity) {
     allEvents.add(eventsOfOneEntity);
   }
 
   /**
-   * Add a list of {@link ATSEventsOfOneEntity} instances into the existing list
+   * Add a list of {@link EventsOfOneEntity} instances into the existing list
    * 
    * @param allEvents
-   *          a list of {@link ATSEventsOfOneEntity} instances
+   *          a list of {@link EventsOfOneEntity} instances
    */
-  public void addEvents(List<ATSEventsOfOneEntity> allEvents) {
+  public void addEvents(List<EventsOfOneEntity> allEvents) {
     this.allEvents.addAll(allEvents);
   }
 
   /**
-   * Set the list to the given list of {@link ATSEventsOfOneEntity} instances
+   * Set the list to the given list of {@link EventsOfOneEntity} instances
    * 
    * @param allEvents
-   *          a list of {@link ATSEventsOfOneEntity} instances
+   *          a list of {@link EventsOfOneEntity} instances
    */
-  public void setEvents(List<ATSEventsOfOneEntity> allEvents) {
+  public void setEvents(List<EventsOfOneEntity> allEvents) {
     this.allEvents.clear();
     this.allEvents.addAll(allEvents);
   }
@@ -94,13 +94,13 @@ public class ATSEvents {
   @XmlAccessorType(XmlAccessType.NONE)
   @Public
   @Unstable
-  public static class ATSEventsOfOneEntity {
+  public static class EventsOfOneEntity {
 
     private String entityId;
     private String entityType;
-    private List<ATSEvent> events = new ArrayList<ATSEvent>();
+    private List<TimelineEvent> events = new ArrayList<TimelineEvent>();
 
-    public ATSEventsOfOneEntity() {
+    public EventsOfOneEntity() {
 
     }
 
@@ -150,7 +150,7 @@ public class ATSEvents {
      * @return a list of events
      */
     @XmlElement(name = "events")
-    public List<ATSEvent> getEvents() {
+    public List<TimelineEvent> getEvents() {
       return events;
     }
 
@@ -160,7 +160,7 @@ public class ATSEvents {
      * @param event
      *          a single event
      */
-    public void addEvent(ATSEvent event) {
+    public void addEvent(TimelineEvent event) {
       events.add(event);
     }
 
@@ -170,7 +170,7 @@ public class ATSEvents {
      * @param events
      *          a list of events
      */
-    public void addEvents(List<ATSEvent> events) {
+    public void addEvents(List<TimelineEvent> events) {
       this.events.addAll(events);
     }
 
@@ -180,7 +180,7 @@ public class ATSEvents {
      * @param events
      *          a list of events
      */
-    public void setEvents(List<ATSEvent> events) {
+    public void setEvents(List<TimelineEvent> events) {
       this.events = events;
     }
 

+ 35 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/apptimeline/ATSPutErrors.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -28,60 +28,60 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * A class that holds a list of put errors.  This is the response returned
- * when a list of {@link ATSEntity} objects is added to the application
- * timeline.  If there are errors in storing individual entity objects,
- * they will be indicated in the list of errors.
+ * A class that holds a list of put errors. This is the response returned when a
+ * list of {@link TimelineEntity} objects is added to the timeline. If there are errors
+ * in storing individual entity objects, they will be indicated in the list of
+ * errors.
  */
-@XmlRootElement(name = "errors")
+@XmlRootElement(name = "response")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
 @Unstable
-public class ATSPutErrors {
+public class TimelinePutResponse {
 
-  private List<ATSPutError> errors = new ArrayList<ATSPutError>();
+  private List<TimelinePutError> errors = new ArrayList<TimelinePutError>();
 
-  public ATSPutErrors() {
+  public TimelinePutResponse() {
 
   }
 
   /**
-   * Get a list of {@link ATSPutError} instances
-   *
-   * @return a list of {@link ATSPutError} instances
+   * Get a list of {@link TimelinePutError} instances
+   * 
+   * @return a list of {@link TimelinePutError} instances
    */
   @XmlElement(name = "errors")
-  public List<ATSPutError> getErrors() {
+  public List<TimelinePutError> getErrors() {
     return errors;
   }
 
   /**
-   * Add a single {@link ATSPutError} instance into the existing list
-   *
+   * Add a single {@link TimelinePutError} instance into the existing list
+   * 
    * @param error
-   *          a single {@link ATSPutError} instance
+   *          a single {@link TimelinePutError} instance
    */
-  public void addError(ATSPutError error) {
+  public void addError(TimelinePutError error) {
     errors.add(error);
   }
 
   /**
-   * Add a list of {@link ATSPutError} instances into the existing list
-   *
+   * Add a list of {@link TimelinePutError} instances into the existing list
+   * 
    * @param errors
-   *          a list of {@link ATSPutError} instances
+   *          a list of {@link TimelinePutError} instances
    */
-  public void addErrors(List<ATSPutError> errors) {
+  public void addErrors(List<TimelinePutError> errors) {
     this.errors.addAll(errors);
   }
 
   /**
-   * Set the list to the given list of {@link ATSPutError} instances
-   *
+   * Set the list to the given list of {@link TimelinePutError} instances
+   * 
    * @param errors
-   *          a list of {@link ATSPutError} instances
+   *          a list of {@link TimelinePutError} instances
    */
-  public void setErrors(List<ATSPutError> errors) {
+  public void setErrors(List<TimelinePutError> errors) {
     this.errors.clear();
     this.errors.addAll(errors);
   }
@@ -93,11 +93,12 @@ public class ATSPutErrors {
   @XmlAccessorType(XmlAccessType.NONE)
   @Public
   @Unstable
-  public static class ATSPutError {
+  public static class TimelinePutError {
+
     /**
      * Error code returned when no start time can be found when putting an
-     * entity. This occurs when the entity does not already exist in the
-     * store and it is put with no start time or events specified.
+     * entity. This occurs when the entity does not already exist in the store
+     * and it is put with no start time or events specified.
      */
     public static final int NO_START_TIME = 1;
     /**
@@ -112,7 +113,7 @@ public class ATSPutErrors {
 
     /**
      * Get the entity Id
-     *
+     * 
      * @return the entity Id
      */
     @XmlElement(name = "entity")
@@ -122,7 +123,7 @@ public class ATSPutErrors {
 
     /**
      * Set the entity Id
-     *
+     * 
      * @param entityId
      *          the entity Id
      */
@@ -132,7 +133,7 @@ public class ATSPutErrors {
 
     /**
      * Get the entity type
-     *
+     * 
      * @return the entity type
      */
     @XmlElement(name = "entitytype")
@@ -142,7 +143,7 @@ public class ATSPutErrors {
 
     /**
      * Set the entity type
-     *
+     * 
      * @param entityType
      *          the entity type
      */
@@ -152,7 +153,7 @@ public class ATSPutErrors {
 
     /**
      * Get the error code
-     *
+     * 
      * @return an error code
      */
     @XmlElement(name = "errorcode")
@@ -162,7 +163,7 @@ public class ATSPutErrors {
 
     /**
      * Set the error code to the given error code
-     *
+     * 
      * @param errorCode
      *          an error code
      */

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/apptimeline/package-info.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java

@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 @InterfaceAudience.Public
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 import org.apache.hadoop.classification.InterfaceAudience;
 

+ 70 - 58
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -300,11 +300,6 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE =
       10;
 
-  /** The implementation class of ApplicationHistoryStore, which is to be used
-   *  by RMApplicationHistoryWriter. */
-  public static final String RM_HISTORY_WRITER_CLASS = RM_PREFIX
-      + "history-writer.class";
-
   //Delegation token related keys
   public static final String  DELEGATION_KEY_UPDATE_INTERVAL_KEY = 
     RM_PREFIX + "delegation.key.update-interval";
@@ -741,7 +736,7 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_MIN_HEALTHY_DISKS_FRACTION =
     NM_PREFIX + "disk-health-checker.min-healthy-disks";
   /**
-   * By default, at least 5% of disks are to be healthy to say that the node
+   * By default, at least 25% of disks are to be healthy to say that the node
    * is healthy in terms of disks.
    */
   public static final float DEFAULT_NM_MIN_HEALTHY_DISKS_FRACTION
@@ -994,74 +989,91 @@ public class YarnConfiguration extends Configuration {
       YARN_PREFIX + "app.container.log.backups";
 
   ////////////////////////////////
-  // AHS Configs
+  // Timeline Service Configs
   ////////////////////////////////
 
-  public static final String AHS_PREFIX = YARN_PREFIX + "ahs.";
+  public static final String TIMELINE_SERVICE_PREFIX =
+      YARN_PREFIX + "timeline-service.";
+
+
+  // mark app-history related configs @Private as application history is going
+  // to be integrated into the timeline service
+  @Private
+  public static final String APPLICATION_HISTORY_PREFIX =
+      TIMELINE_SERVICE_PREFIX + "generic-application-history.";
+
+  /**
+   *  The setting that controls whether application history service is
+   *  enabled or not.
+   */
+  @Private
+  public static final String APPLICATION_HISTORY_ENABLED =
+      APPLICATION_HISTORY_PREFIX + "enabled";
+  @Private
+  public static final boolean DEFAULT_APPLICATION_HISTORY_ENABLED = false;
 
-  /** The setting that controls whether history-service is enabled or not.. */
-  public static final String YARN_HISTORY_SERVICE_ENABLED = AHS_PREFIX
-      + "enabled";
-  public static final boolean DEFAULT_YARN_HISTORY_SERVICE_ENABLED = false;
+  /** Application history store class */
+  @Private
+  public static final String APPLICATION_HISTORY_STORE =
+      APPLICATION_HISTORY_PREFIX + "store-class";
 
   /** URI for FileSystemApplicationHistoryStore */
-  public static final String FS_HISTORY_STORE_URI = AHS_PREFIX + "fs-history-store.uri";
+  @Private
+  public static final String FS_APPLICATION_HISTORY_STORE_URI =
+      APPLICATION_HISTORY_PREFIX + "fs-history-store.uri";
 
   /** T-file compression types used to compress history data.*/
-  public static final String FS_HISTORY_STORE_COMPRESSION_TYPE = AHS_PREFIX + "fs-history-store.compression-type";
-  public static final String DEFAULT_FS_HISTORY_STORE_COMPRESSION_TYPE = "none";
-
-  /** AHS store class */
-  public static final String AHS_STORE = AHS_PREFIX + "store.class";
-
-  /** host:port address for Application History Server API. */
-  public static final String AHS_ADDRESS = AHS_PREFIX + "address";
-  public static final int DEFAULT_AHS_PORT = 10200;
-  public static final String DEFAULT_AHS_ADDRESS = "0.0.0.0:"
-      + DEFAULT_AHS_PORT;
-
-  /** The number of threads to handle client API requests. */
-  public static final String AHS_CLIENT_THREAD_COUNT = AHS_PREFIX
-      + "client.thread-count";
-  public static final int DEFAULT_AHS_CLIENT_THREAD_COUNT = 10;
+  @Private
+  public static final String FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE =
+      APPLICATION_HISTORY_PREFIX + "fs-history-store.compression-type";
+  @Private
+  public static final String DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE =
+      "none";
+
+  /** host:port address for timeline service RPC APIs. */
+  public static final String TIMELINE_SERVICE_ADDRESS =
+      TIMELINE_SERVICE_PREFIX + "address";
+  public static final int DEFAULT_TIMELINE_SERVICE_PORT = 10200;
+  public static final String DEFAULT_TIMELINE_SERVICE_ADDRESS = "0.0.0.0:"
+      + DEFAULT_TIMELINE_SERVICE_PORT;
+
+  /** The number of threads to handle client RPC API requests. */
+  public static final String TIMELINE_SERVICE_HANDLER_THREAD_COUNT =
+      TIMELINE_SERVICE_PREFIX + "handler-thread-count";
+  public static final int DEFAULT_TIMELINE_SERVICE_CLIENT_THREAD_COUNT = 10;
   
 
-  /** The address of the AHS web application.*/
-  public static final String AHS_WEBAPP_ADDRESS = AHS_PREFIX
-      + "webapp.address";
+  /** The address of the timeline service web application.*/
+  public static final String TIMELINE_SERVICE_WEBAPP_ADDRESS =
+      TIMELINE_SERVICE_PREFIX  + "webapp.address";
 
-  public static final int DEFAULT_AHS_WEBAPP_PORT = 8188;
-  public static final String DEFAULT_AHS_WEBAPP_ADDRESS = "0.0.0.0:"
-      + DEFAULT_AHS_WEBAPP_PORT;
+  public static final int DEFAULT_TIMELINE_SERVICE_WEBAPP_PORT = 8188;
+  public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS =
+      "0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_PORT;
 
-  /** The https address of the AHS web application.*/
-  public static final String AHS_WEBAPP_HTTPS_ADDRESS = AHS_PREFIX
-      + "webapp.https.address";
+  /** The https address of the timeline service web application.*/
+  public static final String TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
+      TIMELINE_SERVICE_PREFIX + "webapp.https.address";
 
-  public static final int DEFAULT_AHS_WEBAPP_HTTPS_PORT = 8190;
-  public static final String DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS = "0.0.0.0:"
-      + DEFAULT_AHS_WEBAPP_HTTPS_PORT;
+  public static final int DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT = 8190;
+  public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
+      "0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT;
 
-  /**The kerberos principal to be used for spnego filter for AHS.*/
-  public static final String AHS_WEBAPP_SPNEGO_USER_NAME_KEY =
-      AHS_PREFIX + "webapp.spnego-principal";
-
-  /**The kerberos keytab to be used for spnego filter for AHS.*/
-  public static final String AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
-      AHS_PREFIX + "webapp.spnego-keytab-file";
-
-  ////////////////////////////////
-  // ATS Configs
-  ////////////////////////////////
+  /**The kerberos principal to be used for spnego filter for timeline service.*/
+  public static final String TIMELINE_SERVICE_WEBAPP_SPNEGO_USER_NAME_KEY =
+      TIMELINE_SERVICE_PREFIX + "webapp.spnego-principal";
 
-  public static final String ATS_PREFIX = YARN_PREFIX + "ats.";
+  /**The kerberos keytab to be used for spnego filter for timeline service.*/
+  public static final String TIMELINE_SERVICE_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
+      TIMELINE_SERVICE_PREFIX + "webapp.spnego-keytab-file";
 
-  /** ATS store class */
-  public static final String ATS_STORE = ATS_PREFIX + "store.class";
+  /** Timeline service store class */
+  public static final String TIMELINE_SERVICE_STORE =
+      TIMELINE_SERVICE_PREFIX + "store-class";
 
-  /** ATS leveldb path */
-  public static final String ATS_LEVELDB_PATH_PROPERTY =
-      ATS_PREFIX + "leveldb-apptimeline-store.path";
+  /** Timeline service leveldb path */
+  public static final String TIMELINE_SERVICE_LEVELDB_PATH =
+      TIMELINE_SERVICE_PREFIX + "leveldb-timeline-store.path";
 
   ////////////////////////////////
   // Other Configs

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java

@@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
@@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
  * A client library that can be used to post some information in terms of a
  * number of conceptual entities.
  * 
- * @See ATSEntity
+ * @See Entity
  */
 @Public
 @Unstable
@@ -52,19 +52,19 @@ public abstract class TimelineClient extends AbstractService {
 
   /**
    * <p>
-   * Post the information of a number of conceptual entities of an application
-   * to the timeline server. It is a blocking API. The method will not return
-   * until it gets the response from the timeline server.
+   * Send the information of a number of conceptual entities to the timeline
+   * server. It is a blocking API. The method will not return until it gets the
+   * response from the timeline server.
    * </p>
    * 
    * @param entities
-   *          the collection of {@link ATSEntity}
-   * @return the error information if the post entities are not correctly stored
+   *          the collection of {@link TimelineEntity}
+   * @return the error information if the sent entities are not correctly stored
    * @throws IOException
    * @throws YarnException
    */
   @Public
-  public abstract ATSPutErrors postEntities(
-      ATSEntity... entities) throws IOException, YarnException;
+  public abstract TimelinePutResponse putEntities(
+      TimelineEntity... entities) throws IOException, YarnException;
 
 }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSClientImpl.java

@@ -63,9 +63,9 @@ public class AHSClientImpl extends AHSClient {
   }
 
   private static InetSocketAddress getAHSAddress(Configuration conf) {
-    return conf.getSocketAddr(YarnConfiguration.AHS_ADDRESS,
-        YarnConfiguration.DEFAULT_AHS_ADDRESS,
-        YarnConfiguration.DEFAULT_AHS_PORT);
+    return conf.getSocketAddr(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
   }
 
   @Override

+ 13 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -29,9 +29,9 @@ 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.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -50,7 +50,7 @@ import com.sun.jersey.api.client.config.DefaultClientConfig;
 public class TimelineClientImpl extends TimelineClient {
 
   private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
-  private static final String RESOURCE_URI_STR = "/ws/v1/apptimeline/";
+  private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
   private static final Joiner JOINER = Joiner.on("");
 
   private Client client;
@@ -67,21 +67,21 @@ public class TimelineClientImpl extends TimelineClient {
     if (YarnConfiguration.useHttps(conf)) {
       resURI = URI
           .create(JOINER.join("https://", conf.get(
-              YarnConfiguration.AHS_WEBAPP_HTTPS_ADDRESS,
-              YarnConfiguration.DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS),
+              YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS),
               RESOURCE_URI_STR));
     } else {
       resURI = URI.create(JOINER.join("http://", conf.get(
-          YarnConfiguration.AHS_WEBAPP_ADDRESS,
-          YarnConfiguration.DEFAULT_AHS_WEBAPP_ADDRESS), RESOURCE_URI_STR));
+          YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS), RESOURCE_URI_STR));
     }
     super.serviceInit(conf);
   }
 
   @Override
-  public ATSPutErrors postEntities(
-      ATSEntity... entities) throws IOException, YarnException {
-    ATSEntities entitiesContainer = new ATSEntities();
+  public TimelinePutResponse putEntities(
+      TimelineEntity... entities) throws IOException, YarnException {
+    TimelineEntities entitiesContainer = new TimelineEntities();
     entitiesContainer.addEntities(Arrays.asList(entities));
     ClientResponse resp = doPostingEntities(entitiesContainer);
     if (resp.getClientResponseStatus() != ClientResponse.Status.OK) {
@@ -95,12 +95,12 @@ public class TimelineClientImpl extends TimelineClient {
       }
       throw new YarnException(msg);
     }
-    return resp.getEntity(ATSPutErrors.class);
+    return resp.getEntity(TimelinePutResponse.class);
   }
 
   @Private
   @VisibleForTesting
-  public ClientResponse doPostingEntities(ATSEntities entities) {
+  public ClientResponse doPostingEntities(TimelineEntities entities) {
     WebResource webResource = client.resource(resURI);
     return webResource.accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java

@@ -114,8 +114,8 @@ public class YarnClientImpl extends YarnClient {
         YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
     }
 
-    if (conf.getBoolean(YarnConfiguration.YARN_HISTORY_SERVICE_ENABLED,
-      YarnConfiguration.DEFAULT_YARN_HISTORY_SERVICE_ENABLED)) {
+    if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
+      YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
       historyServiceEnabled = true;
       historyClient = AHSClientImpl.createAHSClient();
       historyClient.init(getConfig());

+ 23 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java

@@ -25,10 +25,10 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 import junit.framework.Assert;
 
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -58,8 +58,8 @@ public class TestTimelineClient {
   public void testPostEntities() throws Exception {
     mockClientResponse(ClientResponse.Status.OK, false);
     try {
-      ATSPutErrors errors = client.postEntities(generateATSEntity());
-      Assert.assertEquals(0, errors.getErrors().size());
+      TimelinePutResponse response = client.putEntities(generateEntity());
+      Assert.assertEquals(0, response.getErrors().size());
     } catch (YarnException e) {
       Assert.fail("Exception is not expected");
     }
@@ -69,14 +69,14 @@ public class TestTimelineClient {
   public void testPostEntitiesWithError() throws Exception {
     mockClientResponse(ClientResponse.Status.OK, true);
     try {
-      ATSPutErrors errors = client.postEntities(generateATSEntity());
-      Assert.assertEquals(1, errors.getErrors().size());
-      Assert.assertEquals("test entity id", errors.getErrors().get(0)
+      TimelinePutResponse response = client.putEntities(generateEntity());
+      Assert.assertEquals(1, response.getErrors().size());
+      Assert.assertEquals("test entity id", response.getErrors().get(0)
           .getEntityId());
-      Assert.assertEquals("test entity type", errors.getErrors().get(0)
+      Assert.assertEquals("test entity type", response.getErrors().get(0)
           .getEntityType());
-      Assert.assertEquals(ATSPutErrors.ATSPutError.IO_EXCEPTION,
-          errors.getErrors().get(0).getErrorCode());
+      Assert.assertEquals(TimelinePutResponse.TimelinePutError.IO_EXCEPTION,
+          response.getErrors().get(0).getErrorCode());
     } catch (YarnException e) {
       Assert.fail("Exception is not expected");
     }
@@ -86,7 +86,7 @@ public class TestTimelineClient {
   public void testPostEntitiesNoResponse() throws Exception {
     mockClientResponse(ClientResponse.Status.INTERNAL_SERVER_ERROR, false);
     try {
-      client.postEntities(generateATSEntity());
+      client.putEntities(generateEntity());
       Assert.fail("Exception is expected");
     } catch (YarnException e) {
       Assert.assertTrue(e.getMessage().contains(
@@ -98,27 +98,28 @@ public class TestTimelineClient {
       boolean hasError) {
     ClientResponse response = mock(ClientResponse.class);
     doReturn(response).when(client)
-        .doPostingEntities(any(ATSEntities.class));
+        .doPostingEntities(any(TimelineEntities.class));
     when(response.getClientResponseStatus()).thenReturn(status);
-    ATSPutErrors.ATSPutError error = new ATSPutErrors.ATSPutError();
+    TimelinePutResponse.TimelinePutError error =
+        new TimelinePutResponse.TimelinePutError();
     error.setEntityId("test entity id");
     error.setEntityType("test entity type");
-    error.setErrorCode(ATSPutErrors.ATSPutError.IO_EXCEPTION);
-    ATSPutErrors errors = new ATSPutErrors();
+    error.setErrorCode(TimelinePutResponse.TimelinePutError.IO_EXCEPTION);
+    TimelinePutResponse putResponse = new TimelinePutResponse();
     if (hasError) {
-      errors.addError(error);
+      putResponse.addError(error);
     }
-    when(response.getEntity(ATSPutErrors.class)).thenReturn(errors);
+    when(response.getEntity(TimelinePutResponse.class)).thenReturn(putResponse);
     return response;
   }
 
-  private static ATSEntity generateATSEntity() {
-    ATSEntity entity = new ATSEntity();
+  private static TimelineEntity generateEntity() {
+    TimelineEntity entity = new TimelineEntity();
     entity.setEntityId("entity id");
     entity.setEntityType("entity type");
     entity.setStartTime(System.currentTimeMillis());
     for (int i = 0; i < 2; ++i) {
-      ATSEvent event = new ATSEvent();
+      TimelineEvent event = new TimelineEvent();
       event.setTimestamp(System.currentTimeMillis());
       event.setEventType("test event type " + i);
       event.addEventInfo("key1", "val1");

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/TimelineUtils.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.util;
+package org.apache.hadoop.yarn.util.timeline;
 
 import java.io.IOException;
 

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/package-info.java

@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+package org.apache.hadoop.yarn.util.timeline;
+import org.apache.hadoop.classification.InterfaceAudience;
+

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java

@@ -142,11 +142,11 @@ public class WebAppUtils {
 
   public static String getAHSWebAppURLWithoutScheme(Configuration conf) {
     if (YarnConfiguration.useHttps(conf)) {
-      return conf.get(YarnConfiguration.AHS_WEBAPP_HTTPS_ADDRESS,
-        YarnConfiguration.DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS);
+      return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
     } else {
-      return conf.get(YarnConfiguration.AHS_WEBAPP_ADDRESS,
-        YarnConfiguration.DEFAULT_AHS_WEBAPP_ADDRESS);
+      return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
     }
   }
   

+ 47 - 57
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -572,30 +572,12 @@
     <value>org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy</value>
   </property>
 
-  <property>
-    <description>Indicate to ResourceManager as well as clients whether
-    history-service is enabled or not. If enabled, ResourceManager starts
-    recording historical data that ApplicationHistory service can consume.
-    Similarly, clients can redirect to the history service when applications
-    finish if this is enabled.</description>
-    <name>yarn.ahs.enabled</name>
-    <value>false</value>
-  </property>
-
   <property>
     <description>Number of worker threads that write the history data.</description>
     <name>yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size</name>
     <value>10</value>
   </property>
 
-  <property>
-    <description>The implementation class of ApplicationHistoryStore, which is
-    to be used by RMApplicationHistoryWriter.
-    </description>
-    <name>yarn.resourcemanager.history-writer.class</name>
-    <value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
-  </property>
-
   <property>
     <description>The class to use as the configuration provider.
     If org.apache.hadoop.yarn.LocalConfigurationProvider is used,
@@ -1085,73 +1067,81 @@
     <value></value>
   </property>
 
-  <!-- Application History Service's Configuration-->
+  <!-- Timeline Service's Configuration-->
 
   <property>
-    <description>The hostname of the AHS.</description>
-    <name>yarn.ahs.hostname</name>
+    <description>The hostname of the timeline service web application.</description>
+    <name>yarn.timeline-service.hostname</name>
     <value>0.0.0.0</value>
   </property>
 
   <property>
-    <description>The http address of the AHS web application.</description>
-    <name>yarn.ahs.webapp.address</name>
-    <value>${yarn.ahs.hostname}:8188</value>
+    <description>This is default address for the timeline server to start the
+    RPC server.</description>
+    <name>yarn.timeline-service.address</name>
+    <value>0.0.0.0:10200</value>
   </property>
 
   <property>
-    <description>The https adddress of the AHS web application.</description>
-    <name>yarn.ahs.webapp.https.address</name>
-    <value>${yarn.ahs.hostname}:8190</value>
+    <description>The http address of the timeline service web application.</description>
+    <name>yarn.timeline-service.webapp.address</name>
+    <value>${yarn.timeline-service.hostname}:8188</value>
   </property>
 
   <property>
-    <description>URI pointing to the location of the FileSystem path where
-    the history will be persisted. This must be supplied when using
-    org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore
-    as the value for yarn.resourcemanager.history-writer.store.class</description>
-    <name>yarn.ahs.fs-history-store.uri</name>
-    <value>${hadoop.log.dir}/yarn/system/ahstore</value>
+    <description>The https adddress of the timeline service web application.</description>
+    <name>yarn.timeline-service.webapp.https.address</name>
+    <value>${yarn.timeline-service.hostname}:8190</value>
   </property>
 
   <property>
-    <description>This is default address for the Application History server
-    to start the RPC server.</description>
-    <name>yarn.ahs.address</name>
-    <value>0.0.0.0:10200</value>
+    <description>Store class name for timeline store</description>
+    <name>yarn.timeline-service.store-class</name>
+    <value>org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.LeveldbTimelineStore</value>
   </property>
 
   <property>
-    <description>CLient thread count to serve the client requests.</description>
-    <name>yarn.ahs.client.thread-count</name>
-    <value>10</value>
+    <description>Store file name for leveldb timeline store</description>
+    <name>yarn.timeline-service.leveldb-timeline-store.path</name>
+    <value>${yarn.log.dir}/timeline</value>
   </property>
-  
+
   <property>
-    <description>T-file compression types used to compress history data.</description>
-    <name>yarn.ahs.fs-history-store.compression-type</name>
-    <value>none</value>
+    <description>Handler thread count to serve the client RPC requests.</description>
+    <name>yarn.timeline-service.handler-thread-count</name>
+    <value>10</value>
   </property>
-  
+
   <property>
-	<description> Store class name for history store, defaulting to file
-		system store </description>
-	<name>yarn.ahs.store.class</name>
-	<value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
+    <description>Indicate to ResourceManager as well as clients whether
+    history-service is enabled or not. If enabled, ResourceManager starts
+    recording historical data that ApplicationHistory service can consume.
+    Similarly, clients can redirect to the history service when applications
+    finish if this is enabled.</description>
+    <name>yarn.timeline-service.generic-application-history.enabled</name>
+    <value>false</value>
   </property>
 
-  <!-- Application Timeline Service's Configuration-->
-
   <property>
-    <description>Store class name for application timeline store</description>
-    <name>yarn.ats.store.class</name>
-    <value>org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.LeveldbApplicationTimelineStore</value>
+    <description>URI pointing to the location of the FileSystem path where
+    the history will be persisted. This must be supplied when using
+    org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore
+    as the value for yarn.timeline-service.generic-application-history.store-class</description>
+    <name>yarn.timeline-service.generic-application-history.fs-history-store.uri</name>
+    <value>${hadoop.log.dir}/yarn/system/history</value>
   </property>
 
   <property>
-    <description>Store file name for leveldb application timeline store</description>
-    <name>yarn.ats.leveldb-apptimeline-store.path</name>
-    <value>${yarn.log.dir}/ats</value>
+    <description>T-file compression types used to compress history data.</description>
+    <name>yarn.timeline-service.generic-application-history.fs-history-store.compression-type</name>
+    <value>none</value>
+  </property>
+  
+  <property>
+	<description>Store class name for history store, defaulting to file
+	system store </description>
+	<name>yarn.timeline-service.generic-application-history.store-class</name>
+	<value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
   </property>
 
   <!-- Other configuration -->

+ 43 - 38
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/apptimeline/TestApplicationTimelineRecords.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timeline/TestTimelineRecords.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.api.records.apptimeline;
+package org.apache.hadoop.yarn.api.records.timeline;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -25,25 +25,30 @@ import junit.framework.Assert;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
-import org.apache.hadoop.yarn.util.TimelineUtils;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Test;
 
-public class TestApplicationTimelineRecords {
+public class TestTimelineRecords {
 
   private static final Log LOG =
-      LogFactory.getLog(TestApplicationTimelineRecords.class);
+      LogFactory.getLog(TestTimelineRecords.class);
 
   @Test
-  public void testATSEntities() throws Exception {
-    ATSEntities entities = new ATSEntities();
+  public void testEntities() throws Exception {
+    TimelineEntities entities = new TimelineEntities();
     for (int j = 0; j < 2; ++j) {
-      ATSEntity entity = new ATSEntity();
+      TimelineEntity entity = new TimelineEntity();
       entity.setEntityId("entity id " + j);
       entity.setEntityType("entity type " + j);
       entity.setStartTime(System.currentTimeMillis());
       for (int i = 0; i < 2; ++i) {
-        ATSEvent event = new ATSEvent();
+        TimelineEvent event = new TimelineEvent();
         event.setTimestamp(System.currentTimeMillis());
         event.setEventType("event type " + i);
         event.addEventInfo("key1", "val1");
@@ -62,14 +67,14 @@ public class TestApplicationTimelineRecords {
     LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(entities, true));
 
     Assert.assertEquals(2, entities.getEntities().size());
-    ATSEntity entity1 = entities.getEntities().get(0);
+    TimelineEntity entity1 = entities.getEntities().get(0);
     Assert.assertEquals("entity id 0", entity1.getEntityId());
     Assert.assertEquals("entity type 0", entity1.getEntityType());
     Assert.assertEquals(2, entity1.getRelatedEntities().size());
     Assert.assertEquals(2, entity1.getEvents().size());
     Assert.assertEquals(2, entity1.getPrimaryFilters().size());
     Assert.assertEquals(2, entity1.getOtherInfo().size());
-    ATSEntity entity2 = entities.getEntities().get(1);
+    TimelineEntity entity2 = entities.getEntities().get(1);
     Assert.assertEquals("entity id 1", entity2.getEntityId());
     Assert.assertEquals("entity type 1", entity2.getEntityType());
     Assert.assertEquals(2, entity2.getRelatedEntities().size());
@@ -79,15 +84,15 @@ public class TestApplicationTimelineRecords {
   }
 
   @Test
-  public void testATSEvents() throws Exception {
-    ATSEvents events = new ATSEvents();
+  public void testEvents() throws Exception {
+    TimelineEvents events = new TimelineEvents();
     for (int j = 0; j < 2; ++j) {
-      ATSEvents.ATSEventsOfOneEntity partEvents =
-          new ATSEvents.ATSEventsOfOneEntity();
+      TimelineEvents.EventsOfOneEntity partEvents =
+          new TimelineEvents.EventsOfOneEntity();
       partEvents.setEntityId("entity id " + j);
       partEvents.setEntityType("entity type " + j);
       for (int i = 0; i < 2; ++i) {
-        ATSEvent event = new ATSEvent();
+        TimelineEvent event = new TimelineEvent();
         event.setTimestamp(System.currentTimeMillis());
         event.setEventType("event type " + i);
         event.addEventInfo("key1", "val1");
@@ -100,57 +105,57 @@ public class TestApplicationTimelineRecords {
     LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(events, true));
 
     Assert.assertEquals(2, events.getAllEvents().size());
-    ATSEvents.ATSEventsOfOneEntity partEvents1 = events.getAllEvents().get(0);
+    TimelineEvents.EventsOfOneEntity partEvents1 = events.getAllEvents().get(0);
     Assert.assertEquals("entity id 0", partEvents1.getEntityId());
     Assert.assertEquals("entity type 0", partEvents1.getEntityType());
     Assert.assertEquals(2, partEvents1.getEvents().size());
-    ATSEvent event11 = partEvents1.getEvents().get(0);
+    TimelineEvent event11 = partEvents1.getEvents().get(0);
     Assert.assertEquals("event type 0", event11.getEventType());
     Assert.assertEquals(2, event11.getEventInfo().size());
-    ATSEvent event12 = partEvents1.getEvents().get(1);
+    TimelineEvent event12 = partEvents1.getEvents().get(1);
     Assert.assertEquals("event type 1", event12.getEventType());
     Assert.assertEquals(2, event12.getEventInfo().size());
-    ATSEvents.ATSEventsOfOneEntity partEvents2 = events.getAllEvents().get(1);
+    TimelineEvents.EventsOfOneEntity partEvents2 = events.getAllEvents().get(1);
     Assert.assertEquals("entity id 1", partEvents2.getEntityId());
     Assert.assertEquals("entity type 1", partEvents2.getEntityType());
     Assert.assertEquals(2, partEvents2.getEvents().size());
-    ATSEvent event21 = partEvents2.getEvents().get(0);
+    TimelineEvent event21 = partEvents2.getEvents().get(0);
     Assert.assertEquals("event type 0", event21.getEventType());
     Assert.assertEquals(2, event21.getEventInfo().size());
-    ATSEvent event22 = partEvents2.getEvents().get(1);
+    TimelineEvent event22 = partEvents2.getEvents().get(1);
     Assert.assertEquals("event type 1", event22.getEventType());
     Assert.assertEquals(2, event22.getEventInfo().size());
   }
 
   @Test
-  public void testATSPutErrors() throws Exception {
-    ATSPutErrors atsPutErrors = new ATSPutErrors();
-    ATSPutError error1 = new ATSPutError();
+  public void testTimelinePutErrors() throws Exception {
+    TimelinePutResponse TimelinePutErrors = new TimelinePutResponse();
+    TimelinePutError error1 = new TimelinePutError();
     error1.setEntityId("entity id 1");
     error1.setEntityId("entity type 1");
-    error1.setErrorCode(ATSPutError.NO_START_TIME);
-    atsPutErrors.addError(error1);
-    List<ATSPutError> errors = new ArrayList<ATSPutError>();
-    errors.add(error1);
-    ATSPutError error2 = new ATSPutError();
+    error1.setErrorCode(TimelinePutError.NO_START_TIME);
+    TimelinePutErrors.addError(error1);
+    List<TimelinePutError> response = new ArrayList<TimelinePutError>();
+    response.add(error1);
+    TimelinePutError error2 = new TimelinePutError();
     error2.setEntityId("entity id 2");
     error2.setEntityId("entity type 2");
-    error2.setErrorCode(ATSPutError.IO_EXCEPTION);
-    errors.add(error2);
-    atsPutErrors.addErrors(errors);
+    error2.setErrorCode(TimelinePutError.IO_EXCEPTION);
+    response.add(error2);
+    TimelinePutErrors.addErrors(response);
     LOG.info("Errors in JSON:");
-    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(atsPutErrors, true));
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(TimelinePutErrors, true));
 
-    Assert.assertEquals(3, atsPutErrors.getErrors().size());
-    ATSPutError e = atsPutErrors.getErrors().get(0);
+    Assert.assertEquals(3, TimelinePutErrors.getErrors().size());
+    TimelinePutError e = TimelinePutErrors.getErrors().get(0);
     Assert.assertEquals(error1.getEntityId(), e.getEntityId());
     Assert.assertEquals(error1.getEntityType(), e.getEntityType());
     Assert.assertEquals(error1.getErrorCode(), e.getErrorCode());
-    e = atsPutErrors.getErrors().get(1);
+    e = TimelinePutErrors.getErrors().get(1);
     Assert.assertEquals(error1.getEntityId(), e.getEntityId());
     Assert.assertEquals(error1.getEntityType(), e.getEntityType());
     Assert.assertEquals(error1.getErrorCode(), e.getErrorCode());
-    e = atsPutErrors.getErrors().get(2);
+    e = TimelinePutErrors.getErrors().get(2);
     Assert.assertEquals(error2.getEntityId(), e.getEntityId());
     Assert.assertEquals(error2.getEntityType(), e.getEntityType());
     Assert.assertEquals(error2.getErrorCode(), e.getErrorCode());

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java

@@ -76,19 +76,19 @@ public class ApplicationHistoryClientService extends AbstractService {
     Configuration conf = getConfig();
     YarnRPC rpc = YarnRPC.create(conf);
     InetSocketAddress address =
-        conf.getSocketAddr(YarnConfiguration.AHS_ADDRESS,
-          YarnConfiguration.DEFAULT_AHS_ADDRESS,
-          YarnConfiguration.DEFAULT_AHS_PORT);
+        conf.getSocketAddr(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
 
     server =
         rpc.getServer(ApplicationHistoryProtocol.class, protocolHandler,
           address, conf, null, conf.getInt(
-            YarnConfiguration.AHS_CLIENT_THREAD_COUNT,
-            YarnConfiguration.DEFAULT_AHS_CLIENT_THREAD_COUNT));
+            YarnConfiguration.TIMELINE_SERVICE_HANDLER_THREAD_COUNT,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_THREAD_COUNT));
 
     server.start();
     this.bindAddress =
-        conf.updateConnectAddr(YarnConfiguration.AHS_ADDRESS,
+        conf.updateConnectAddr(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
           server.getListenerAddress());
     LOG.info("Instantiated ApplicationHistoryClientService at "
         + this.bindAddress);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java

@@ -79,7 +79,8 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
   protected ApplicationHistoryStore createApplicationHistoryStore(
       Configuration conf) {
     return ReflectionUtils.newInstance(conf.getClass(
-      YarnConfiguration.AHS_STORE, FileSystemApplicationHistoryStore.class,
+      YarnConfiguration.APPLICATION_HISTORY_STORE,
+      FileSystemApplicationHistoryStore.class,
       ApplicationHistoryStore.class), conf);
   }
 

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java

@@ -33,8 +33,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.LeveldbApplicationTimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.LeveldbTimelineStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
@@ -54,7 +54,7 @@ public class ApplicationHistoryServer extends CompositeService {
 
   ApplicationHistoryClientService ahsClientService;
   ApplicationHistoryManager historyManager;
-  ApplicationTimelineStore timelineStore;
+  TimelineStore timelineStore;
   private WebApp webApp;
 
   public ApplicationHistoryServer() {
@@ -67,7 +67,7 @@ public class ApplicationHistoryServer extends CompositeService {
     ahsClientService = createApplicationHistoryClientService(historyManager);
     addService(ahsClientService);
     addService((Service) historyManager);
-    timelineStore = createApplicationTimelineStore(conf);
+    timelineStore = createTimelineStore(conf);
     addIfService(timelineStore);
     super.serviceInit(conf);
   }
@@ -141,11 +141,11 @@ public class ApplicationHistoryServer extends CompositeService {
     return new ApplicationHistoryManagerImpl();
   }
 
-  protected ApplicationTimelineStore createApplicationTimelineStore(
+  protected TimelineStore createTimelineStore(
       Configuration conf) {
     return ReflectionUtils.newInstance(conf.getClass(
-        YarnConfiguration.ATS_STORE, LeveldbApplicationTimelineStore.class,
-        ApplicationTimelineStore.class), conf);
+        YarnConfiguration.TIMELINE_SERVICE_STORE, LeveldbTimelineStore.class,
+        TimelineStore.class), conf);
   }
 
   protected void startWebApp() {
@@ -158,9 +158,9 @@ public class ApplicationHistoryServer extends CompositeService {
               ahsClientService, "ws")
             .with(getConfig())
             .withHttpSpnegoPrincipalKey(
-              YarnConfiguration.AHS_WEBAPP_SPNEGO_USER_NAME_KEY)
+              YarnConfiguration.TIMELINE_SERVICE_WEBAPP_SPNEGO_USER_NAME_KEY)
             .withHttpSpnegoKeytabKey(
-              YarnConfiguration.AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
+              YarnConfiguration.TIMELINE_SERVICE_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
             .at(bindAddress)
             .start(new AHSWebApp(historyManager, timelineStore));
     } catch (Exception e) {

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java

@@ -113,7 +113,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
   @Override
   public void serviceInit(Configuration conf) throws Exception {
     Path fsWorkingPath =
-        new Path(conf.get(YarnConfiguration.FS_HISTORY_STORE_URI));
+        new Path(conf.get(YarnConfiguration.FS_APPLICATION_HISTORY_STORE_URI));
     rootDirPath = new Path(fsWorkingPath, ROOT_DIR_NAME);
     try {
       fs = fsWorkingPath.getFileSystem(conf);
@@ -727,8 +727,8 @@ public class FileSystemApplicationHistoryStore extends AbstractService
       fs.setPermission(historyFile, HISTORY_FILE_UMASK);
       writer =
           new TFile.Writer(fsdos, MIN_BLOCK_SIZE, getConfig().get(
-            YarnConfiguration.FS_HISTORY_STORE_COMPRESSION_TYPE,
-            YarnConfiguration.DEFAULT_FS_HISTORY_STORE_COMPRESSION_TYPE), null,
+            YarnConfiguration.FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE,
+            YarnConfiguration.DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE), null,
             getConfig());
     }
 

+ 0 - 131
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineReader.java

@@ -1,131 +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.applicationhistoryservice.apptimeline;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Set;
-import java.util.SortedSet;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
-
-/**
- * This interface is for retrieving application timeline information.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public interface ApplicationTimelineReader {
-
-  /**
-   * Possible fields to retrieve for {@link #getEntities} and {@link
-   * #getEntity}.
-   */
-  enum Field {
-    EVENTS,
-    RELATED_ENTITIES,
-    PRIMARY_FILTERS,
-    OTHER_INFO,
-    LAST_EVENT_ONLY
-  }
-
-  /**
-   * Default limit for {@link #getEntities} and {@link #getEntityTimelines}.
-   */
-  final long DEFAULT_LIMIT = 100;
-
-  /**
-   * This method retrieves a list of entity information, {@link ATSEntity},
-   * sorted by the starting timestamp for the entity, descending.
-   *
-   * @param entityType The type of entities to return (required).
-   * @param limit A limit on the number of entities to return. If null,
-   *              defaults to {@link #DEFAULT_LIMIT}.
-   * @param windowStart The earliest start timestamp to retrieve (exclusive).
-   *                    If null, defaults to retrieving all entities until the
-   *                    limit is reached.
-   * @param windowEnd The latest start timestamp to retrieve (inclusive).
-   *                  If null, defaults to {@link Long#MAX_VALUE}
-   * @param primaryFilter Retrieves only entities that have the specified
-   *                      primary filter. If null, retrieves all entities.
-   *                      This is an indexed retrieval, and no entities that
-   *                      do not match the filter are scanned.
-   * @param secondaryFilters Retrieves only entities that have exact matches
-   *                         for all the specified filters in their primary
-   *                         filters or other info. This is not an indexed
-   *                         retrieval, so all entities are scanned but only
-   *                         those matching the filters are returned.
-   * @param fieldsToRetrieve Specifies which fields of the entity object to
-   *                         retrieve (see {@link Field}). If the set of fields
-   *                         contains {@link Field#LAST_EVENT_ONLY} and not
-   *                         {@link Field#EVENTS}, the most recent event for
-   *                         each entity is retrieved. If null, retrieves all
-   *                         fields.
-   * @return An {@link ATSEntities} object.
-   * @throws IOException
-   */
-  ATSEntities getEntities(String entityType,
-      Long limit, Long windowStart, Long windowEnd,
-      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fieldsToRetrieve) throws IOException;
-
-  /**
-   * This method retrieves the entity information for a given entity.
-   *
-   * @param entity The entity whose information will be retrieved.
-   * @param entityType The type of the entity.
-   * @param fieldsToRetrieve Specifies which fields of the entity object to
-   *                         retrieve (see {@link Field}). If the set of
-   *                         fields contains {@link Field#LAST_EVENT_ONLY} and
-   *                         not {@link Field#EVENTS}, the most recent event
-   *                         for each entity is retrieved. If null, retrieves
-   *                         all fields.
-   * @return An {@link ATSEntity} object.
-   * @throws IOException
-   */
-  ATSEntity getEntity(String entity, String entityType, EnumSet<Field>
-      fieldsToRetrieve) throws IOException;
-
-  /**
-   * This method retrieves the events for a list of entities all of the same
-   * entity type. The events for each entity are sorted in order of their
-   * timestamps, descending.
-   *
-   * @param entityType The type of entities to retrieve events for.
-   * @param entityIds The entity IDs to retrieve events for.
-   * @param limit A limit on the number of events to return for each entity.
-   *              If null, defaults to  {@link #DEFAULT_LIMIT} events per
-   *              entity.
-   * @param windowStart If not null, retrieves only events later than the
-   *                    given time (exclusive)
-   * @param windowEnd If not null, retrieves only events earlier than the
-   *                  given time (inclusive)
-   * @param eventTypes Restricts the events returned to the given types. If
-   *                   null, events of all types will be returned.
-   * @return An {@link ATSEvents} object.
-   * @throws IOException
-   */
-  ATSEvents getEntityTimelines(String entityType,
-      SortedSet<String> entityIds, Long limit, Long windowStart,
-      Long windowEnd, Set<String> eventTypes) throws IOException;
-}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/EntityIdentifier.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/EntityIdentifier.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;

+ 24 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/GenericObjectMapper.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/GenericObjectMapper.java

@@ -15,12 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.WritableUtils;
-import org.codehaus.jackson.map.ObjectMapper;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -31,14 +26,18 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.WritableUtils;
+import org.codehaus.jackson.map.ObjectMapper;
 
 /**
  * A utility class providing methods for serializing and deserializing
  * objects. The {@link #write(Object)}, {@link #read(byte[])} and {@link
  * #write(java.io.DataOutputStream, Object)}, {@link
  * #read(java.io.DataInputStream)} methods are used by the
- * {@link LeveldbApplicationTimelineStore} to store and retrieve arbitrary
+ * {@link LeveldbTimelineStore} to store and retrieve arbitrary
  * JSON, while the {@link #writeReverseOrderedLong} and {@link
  * #readReverseOrderedLong} methods are used to sort entities in descending
  * start time order.
@@ -132,9 +131,24 @@ public class GenericObjectMapper {
    * @throws IOException
    */
   public static Object read(byte[] b) throws IOException {
-    if (b == null || b.length == 0)
+    return read(b, 0);
+  }
+
+  /**
+   * Deserializes an Object from a byte array at a specified offset, assuming
+   * the bytes were created with {@link #write(Object)}.
+   *
+   * @param b A byte array
+   * @param offset Offset into the array
+   * @return An Object
+   * @throws IOException
+   */
+  public static Object read(byte[] b, int offset) throws IOException {
+    if (b == null || b.length == 0) {
       return null;
-    ByteArrayInputStream bais = new ByteArrayInputStream(b);
+    }
+    ByteArrayInputStream bais = new ByteArrayInputStream(b, offset,
+        b.length - offset);
     return read(new DataInputStream(bais));
   }
 

+ 144 - 125
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/LeveldbApplicationTimelineStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/LeveldbTimelineStore.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -44,13 +44,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents.ATSEventsOfOneEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
@@ -58,22 +58,20 @@ import org.iq80.leveldb.DBIterator;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.WriteBatch;
 
-import static org.apache.hadoop.yarn.server.applicationhistoryservice
-    .apptimeline.GenericObjectMapper.readReverseOrderedLong;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice
-    .apptimeline.GenericObjectMapper.writeReverseOrderedLong;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper.readReverseOrderedLong;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper.writeReverseOrderedLong;
 
 /**
- * An implementation of an application timeline store backed by leveldb.
+ * An implementation of a timeline store backed by leveldb.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class LeveldbApplicationTimelineStore extends AbstractService
-    implements ApplicationTimelineStore {
+public class LeveldbTimelineStore extends AbstractService
+    implements TimelineStore {
   private static final Log LOG = LogFactory
-      .getLog(LeveldbApplicationTimelineStore.class);
+      .getLog(LeveldbTimelineStore.class);
 
-  private static final String FILENAME = "leveldb-apptimeline-store.ldb";
+  private static final String FILENAME = "leveldb-timeline-store.ldb";
 
   private static final byte[] START_TIME_LOOKUP_PREFIX = "k".getBytes();
   private static final byte[] ENTITY_ENTRY_PREFIX = "e".getBytes();
@@ -94,8 +92,8 @@ public class LeveldbApplicationTimelineStore extends AbstractService
 
   private DB db;
 
-  public LeveldbApplicationTimelineStore() {
-    super(LeveldbApplicationTimelineStore.class.getName());
+  public LeveldbTimelineStore() {
+    super(LeveldbTimelineStore.class.getName());
   }
 
   @Override
@@ -103,12 +101,12 @@ public class LeveldbApplicationTimelineStore extends AbstractService
     Options options = new Options();
     options.createIfMissing(true);
     JniDBFactory factory = new JniDBFactory();
-    String path = conf.get(YarnConfiguration.ATS_LEVELDB_PATH_PROPERTY);
+    String path = conf.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH);
     File p = new File(path);
     if (!p.exists())
       if (!p.mkdirs())
         throw new IOException("Couldn't create directory for leveldb " +
-            "application timeline store " + path);
+            "timeline store " + path);
     LOG.info("Using leveldb path " + path);
     db = factory.open(new File(path, FILENAME), options);
     super.serviceInit(conf);
@@ -212,20 +210,20 @@ public class LeveldbApplicationTimelineStore extends AbstractService
   }
 
   @Override
-  public ATSEntity getEntity(String entity, String entityType,
+  public TimelineEntity getEntity(String entityId, String entityType,
       EnumSet<Field> fields) throws IOException {
     DBIterator iterator = null;
     try {
-      byte[] revStartTime = getStartTime(entity, entityType, null, null, null);
+      byte[] revStartTime = getStartTime(entityId, entityType, null, null, null);
       if (revStartTime == null)
         return null;
       byte[] prefix = KeyBuilder.newInstance().add(ENTITY_ENTRY_PREFIX)
-          .add(entityType).add(revStartTime).add(entity).getBytesForLookup();
+          .add(entityType).add(revStartTime).add(entityId).getBytesForLookup();
 
       iterator = db.iterator();
       iterator.seek(prefix);
 
-      return getEntity(entity, entityType,
+      return getEntity(entityId, entityType,
           readReverseOrderedLong(revStartTime, 0), fields, iterator, prefix,
           prefix.length);
     } finally {
@@ -237,45 +235,43 @@ public class LeveldbApplicationTimelineStore extends AbstractService
    * Read entity from a db iterator.  If no information is found in the
    * specified fields for this entity, return null.
    */
-  private static ATSEntity getEntity(String entity, String entityType,
+  private static TimelineEntity getEntity(String entityId, String entityType,
       Long startTime, EnumSet<Field> fields, DBIterator iterator,
       byte[] prefix, int prefixlen) throws IOException {
     if (fields == null)
       fields = EnumSet.allOf(Field.class);
 
-    ATSEntity atsEntity = new ATSEntity();
+    TimelineEntity entity = new TimelineEntity();
     boolean events = false;
     boolean lastEvent = false;
     if (fields.contains(Field.EVENTS)) {
       events = true;
-      atsEntity.setEvents(new ArrayList<ATSEvent>());
+      entity.setEvents(new ArrayList<TimelineEvent>());
     } else if (fields.contains(Field.LAST_EVENT_ONLY)) {
       lastEvent = true;
-      atsEntity.setEvents(new ArrayList<ATSEvent>());
+      entity.setEvents(new ArrayList<TimelineEvent>());
     }
     else {
-      atsEntity.setEvents(null);
+      entity.setEvents(null);
     }
     boolean relatedEntities = false;
     if (fields.contains(Field.RELATED_ENTITIES)) {
       relatedEntities = true;
-      atsEntity.setRelatedEntities(new HashMap<String, List<String>>());
     } else {
-      atsEntity.setRelatedEntities(null);
+      entity.setRelatedEntities(null);
     }
     boolean primaryFilters = false;
     if (fields.contains(Field.PRIMARY_FILTERS)) {
       primaryFilters = true;
-      atsEntity.setPrimaryFilters(new HashMap<String, Object>());
     } else {
-      atsEntity.setPrimaryFilters(null);
+      entity.setPrimaryFilters(null);
     }
     boolean otherInfo = false;
     if (fields.contains(Field.OTHER_INFO)) {
       otherInfo = true;
-      atsEntity.setOtherInfo(new HashMap<String, Object>());
+      entity.setOtherInfo(new HashMap<String, Object>());
     } else {
-      atsEntity.setOtherInfo(null);
+      entity.setOtherInfo(null);
     }
 
     // iterate through the entity's entry, parsing information if it is part
@@ -286,49 +282,48 @@ public class LeveldbApplicationTimelineStore extends AbstractService
         break;
       if (key[prefixlen] == PRIMARY_FILTER_COLUMN[0]) {
         if (primaryFilters) {
-          atsEntity.addPrimaryFilter(parseRemainingKey(key,
-              prefixlen + PRIMARY_FILTER_COLUMN.length),
-              GenericObjectMapper.read(iterator.peekNext().getValue()));
+          addPrimaryFilter(entity, key,
+              prefixlen + PRIMARY_FILTER_COLUMN.length);
         }
       } else if (key[prefixlen] == OTHER_INFO_COLUMN[0]) {
         if (otherInfo) {
-          atsEntity.addOtherInfo(parseRemainingKey(key,
+          entity.addOtherInfo(parseRemainingKey(key,
               prefixlen + OTHER_INFO_COLUMN.length),
               GenericObjectMapper.read(iterator.peekNext().getValue()));
         }
       } else if (key[prefixlen] == RELATED_COLUMN[0]) {
         if (relatedEntities) {
-          addRelatedEntity(atsEntity, key,
+          addRelatedEntity(entity, key,
               prefixlen + RELATED_COLUMN.length);
         }
       } else if (key[prefixlen] == TIME_COLUMN[0]) {
-        if (events || (lastEvent && atsEntity.getEvents().size() == 0)) {
-          ATSEvent event = getEntityEvent(null, key, prefixlen +
+        if (events || (lastEvent && entity.getEvents().size() == 0)) {
+          TimelineEvent event = getEntityEvent(null, key, prefixlen +
               TIME_COLUMN.length, iterator.peekNext().getValue());
           if (event != null) {
-            atsEntity.addEvent(event);
+            entity.addEvent(event);
           }
         }
       } else {
         LOG.warn(String.format("Found unexpected column for entity %s of " +
-            "type %s (0x%02x)", entity, entityType, key[prefixlen]));
+            "type %s (0x%02x)", entityId, entityType, key[prefixlen]));
       }
     }
 
-    atsEntity.setEntityId(entity);
-    atsEntity.setEntityType(entityType);
-    atsEntity.setStartTime(startTime);
+    entity.setEntityId(entityId);
+    entity.setEntityType(entityType);
+    entity.setStartTime(startTime);
 
-    return atsEntity;
+    return entity;
   }
 
   @Override
-  public ATSEvents getEntityTimelines(String entityType,
+  public TimelineEvents getEntityTimelines(String entityType,
       SortedSet<String> entityIds, Long limit, Long windowStart,
       Long windowEnd, Set<String> eventType) throws IOException {
-    ATSEvents atsEvents = new ATSEvents();
+    TimelineEvents events = new TimelineEvents();
     if (entityIds == null || entityIds.isEmpty())
-      return atsEvents;
+      return events;
     // create a lexicographically-ordered map from start time to entities
     Map<byte[], List<EntityIdentifier>> startTimeMap = new TreeMap<byte[],
         List<EntityIdentifier>>(new Comparator<byte[]>() {
@@ -359,13 +354,13 @@ public class LeveldbApplicationTimelineStore extends AbstractService
         // start time, end time, event types) for entities whose start times
         // were found and add the entities to the return list
         byte[] revStartTime = entry.getKey();
-        for (EntityIdentifier entity : entry.getValue()) {
-          ATSEventsOfOneEntity atsEntity = new ATSEventsOfOneEntity();
-          atsEntity.setEntityId(entity.getId());
-          atsEntity.setEntityType(entityType);
-          atsEvents.addEvent(atsEntity);
+        for (EntityIdentifier entityID : entry.getValue()) {
+          EventsOfOneEntity entity = new EventsOfOneEntity();
+          entity.setEntityId(entityID.getId());
+          entity.setEntityType(entityType);
+          events.addEvent(entity);
           KeyBuilder kb = KeyBuilder.newInstance().add(ENTITY_ENTRY_PREFIX)
-              .add(entityType).add(revStartTime).add(entity.getId())
+              .add(entityType).add(revStartTime).add(entityID.getId())
               .add(TIME_COLUMN);
           byte[] prefix = kb.getBytesForLookup();
           if (windowEnd == null) {
@@ -383,24 +378,24 @@ public class LeveldbApplicationTimelineStore extends AbstractService
             limit = DEFAULT_LIMIT;
           }
           iterator = db.iterator();
-          for (iterator.seek(first); atsEntity.getEvents().size() < limit &&
+          for (iterator.seek(first); entity.getEvents().size() < limit &&
               iterator.hasNext(); iterator.next()) {
             byte[] key = iterator.peekNext().getKey();
             if (!prefixMatches(prefix, prefix.length, key) || (last != null &&
                 WritableComparator.compareBytes(key, 0, key.length, last, 0,
                     last.length) > 0))
               break;
-            ATSEvent event = getEntityEvent(eventType, key, prefix.length,
+            TimelineEvent event = getEntityEvent(eventType, key, prefix.length,
                 iterator.peekNext().getValue());
             if (event != null)
-              atsEntity.addEvent(event);
+              entity.addEvent(event);
           }
         }
       }
     } finally {
       IOUtils.cleanup(LOG, iterator);
     }
-    return atsEvents;
+    return events;
   }
 
   /**
@@ -415,7 +410,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
   }
 
   @Override
-  public ATSEntities getEntities(String entityType,
+  public TimelineEntities getEntities(String entityType,
       Long limit, Long windowStart, Long windowEnd,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
       EnumSet<Field> fields) throws IOException {
@@ -450,7 +445,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
    * @return A list of entities
    * @throws IOException
    */
-  private ATSEntities getEntityByTime(byte[] base,
+  private TimelineEntities getEntityByTime(byte[] base,
       String entityType, Long limit, Long starttime, Long endtime,
       Collection<NameValuePair> secondaryFilters, EnumSet<Field> fields)
       throws IOException {
@@ -479,82 +474,87 @@ public class LeveldbApplicationTimelineStore extends AbstractService
         limit = DEFAULT_LIMIT;
       }
 
-      ATSEntities atsEntities = new ATSEntities();
+      TimelineEntities entities = new TimelineEntities();
       iterator = db.iterator();
       iterator.seek(first);
       // iterate until one of the following conditions is met: limit is
       // reached, there are no more keys, the key prefix no longer matches,
       // or a start time has been specified and reached/exceeded
-      while (atsEntities.getEntities().size() < limit && iterator.hasNext()) {
+      while (entities.getEntities().size() < limit && iterator.hasNext()) {
         byte[] key = iterator.peekNext().getKey();
         if (!prefixMatches(prefix, prefix.length, key) || (last != null &&
             WritableComparator.compareBytes(key, 0, key.length, last, 0,
                 last.length) > 0))
           break;
-        // read the start time and entity from the current key
+        // read the start time and entityId from the current key
         KeyParser kp = new KeyParser(key, prefix.length);
         Long startTime = kp.getNextLong();
-        String entity = kp.getNextString();
+        String entityId = kp.getNextString();
         // parse the entity that owns this key, iterating over all keys for
         // the entity
-        ATSEntity atsEntity = getEntity(entity, entityType, startTime,
+        TimelineEntity entity = getEntity(entityId, entityType, startTime,
             fields, iterator, key, kp.getOffset());
-        if (atsEntity == null)
+        if (entity == null)
           continue;
         // determine if the retrieved entity matches the provided secondary
         // filters, and if so add it to the list of entities to return
         boolean filterPassed = true;
         if (secondaryFilters != null) {
           for (NameValuePair filter : secondaryFilters) {
-            Object v = atsEntity.getOtherInfo().get(filter.getName());
-            if (v == null)
-              v = atsEntity.getPrimaryFilters().get(filter.getName());
-            if (v == null || !v.equals(filter.getValue())) {
+            Object v = entity.getOtherInfo().get(filter.getName());
+            if (v == null) {
+              Set<Object> vs = entity.getPrimaryFilters()
+                  .get(filter.getName());
+              if (vs != null && !vs.contains(filter.getValue())) {
+                filterPassed = false;
+                break;
+              }
+            } else if (!v.equals(filter.getValue())) {
               filterPassed = false;
               break;
             }
           }
         }
         if (filterPassed)
-          atsEntities.addEntity(atsEntity);
+          entities.addEntity(entity);
       }
-      return atsEntities;
+      return entities;
     } finally {
       IOUtils.cleanup(LOG, iterator);
     }
   }
 
   /**
-   * Put a single entity.  If there is an error, add a PutError to the given
+   * Put a single entity.  If there is an error, add a TimelinePutError to the given
    * response.
    */
-  private void put(ATSEntity atsEntity, ATSPutErrors response) {
+  private void put(TimelineEntity entity, TimelinePutResponse response) {
     WriteBatch writeBatch = null;
     try {
       writeBatch = db.createWriteBatch();
-      List<ATSEvent> events = atsEntity.getEvents();
+      List<TimelineEvent> events = entity.getEvents();
       // look up the start time for the entity
-      byte[] revStartTime = getStartTime(atsEntity.getEntityId(),
-          atsEntity.getEntityType(), atsEntity.getStartTime(), events,
+      byte[] revStartTime = getStartTime(entity.getEntityId(),
+          entity.getEntityType(), entity.getStartTime(), events,
           writeBatch);
       if (revStartTime == null) {
         // if no start time is found, add an error and return
-        ATSPutError error = new ATSPutError();
-        error.setEntityId(atsEntity.getEntityId());
-        error.setEntityType(atsEntity.getEntityType());
-        error.setErrorCode(ATSPutError.NO_START_TIME);
+        TimelinePutError error = new TimelinePutError();
+        error.setEntityId(entity.getEntityId());
+        error.setEntityType(entity.getEntityType());
+        error.setErrorCode(TimelinePutError.NO_START_TIME);
         response.addError(error);
         return;
       }
       Long revStartTimeLong = readReverseOrderedLong(revStartTime, 0);
-      Map<String, Object> primaryFilters = atsEntity.getPrimaryFilters();
+      Map<String, Set<Object>> primaryFilters = entity.getPrimaryFilters();
 
       // write event entries
       if (events != null && !events.isEmpty()) {
-        for (ATSEvent event : events) {
+        for (TimelineEvent event : events) {
           byte[] revts = writeReverseOrderedLong(event.getTimestamp());
-          byte[] key = createEntityEventKey(atsEntity.getEntityId(),
-              atsEntity.getEntityType(), revStartTime, revts,
+          byte[] key = createEntityEventKey(entity.getEntityId(),
+              entity.getEntityType(), revStartTime, revts,
               event.getEventType());
           byte[] value = GenericObjectMapper.write(event.getEventInfo());
           writeBatch.put(key, value);
@@ -563,10 +563,10 @@ public class LeveldbApplicationTimelineStore extends AbstractService
       }
 
       // write related entity entries
-      Map<String,List<String>> relatedEntities =
-          atsEntity.getRelatedEntities();
+      Map<String, Set<String>> relatedEntities =
+          entity.getRelatedEntities();
       if (relatedEntities != null && !relatedEntities.isEmpty()) {
-        for (Entry<String, List<String>> relatedEntityList :
+        for (Entry<String, Set<String>> relatedEntityList :
             relatedEntities.entrySet()) {
           String relatedEntityType = relatedEntityList.getKey();
           for (String relatedEntityId : relatedEntityList.getValue()) {
@@ -586,7 +586,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
             // write reverse entry (related entity -> entity)
             byte[] key = createReleatedEntityKey(relatedEntityId,
                 relatedEntityType, relatedEntityStartTime,
-                atsEntity.getEntityId(), atsEntity.getEntityType());
+                entity.getEntityId(), entity.getEntityType());
             writeBatch.put(key, EMPTY_BYTES);
             // TODO: write forward entry (entity -> related entity)?
           }
@@ -595,21 +595,25 @@ public class LeveldbApplicationTimelineStore extends AbstractService
 
       // write primary filter entries
       if (primaryFilters != null && !primaryFilters.isEmpty()) {
-        for (Entry<String, Object> primaryFilter : primaryFilters.entrySet()) {
-          byte[] key = createPrimaryFilterKey(atsEntity.getEntityId(),
-              atsEntity.getEntityType(), revStartTime, primaryFilter.getKey());
-          byte[] value = GenericObjectMapper.write(primaryFilter.getValue());
-          writeBatch.put(key, value);
-          writePrimaryFilterEntries(writeBatch, primaryFilters, key, value);
+        for (Entry<String, Set<Object>> primaryFilter :
+            primaryFilters.entrySet()) {
+          for (Object primaryFilterValue : primaryFilter.getValue()) {
+            byte[] key = createPrimaryFilterKey(entity.getEntityId(),
+                entity.getEntityType(), revStartTime,
+                primaryFilter.getKey(), primaryFilterValue);
+            writeBatch.put(key, EMPTY_BYTES);
+            writePrimaryFilterEntries(writeBatch, primaryFilters, key,
+                EMPTY_BYTES);
+          }
         }
       }
 
       // write other info entries
-      Map<String, Object> otherInfo = atsEntity.getOtherInfo();
+      Map<String, Object> otherInfo = entity.getOtherInfo();
       if (otherInfo != null && !otherInfo.isEmpty()) {
         for (Entry<String, Object> i : otherInfo.entrySet()) {
-          byte[] key = createOtherInfoKey(atsEntity.getEntityId(),
-              atsEntity.getEntityType(), revStartTime, i.getKey());
+          byte[] key = createOtherInfoKey(entity.getEntityId(),
+              entity.getEntityType(), revStartTime, i.getKey());
           byte[] value = GenericObjectMapper.write(i.getValue());
           writeBatch.put(key, value);
           writePrimaryFilterEntries(writeBatch, primaryFilters, key, value);
@@ -617,12 +621,12 @@ public class LeveldbApplicationTimelineStore extends AbstractService
       }
       db.write(writeBatch);
     } catch (IOException e) {
-      LOG.error("Error putting entity " + atsEntity.getEntityId() +
-          " of type " + atsEntity.getEntityType(), e);
-      ATSPutError error = new ATSPutError();
-      error.setEntityId(atsEntity.getEntityId());
-      error.setEntityType(atsEntity.getEntityType());
-      error.setErrorCode(ATSPutError.IO_EXCEPTION);
+      LOG.error("Error putting entity " + entity.getEntityId() +
+          " of type " + entity.getEntityType(), e);
+      TimelinePutError error = new TimelinePutError();
+      error.setEntityId(entity.getEntityId());
+      error.setEntityType(entity.getEntityType());
+      error.setErrorCode(TimelinePutError.IO_EXCEPTION);
       response.addError(error);
     } finally {
       IOUtils.cleanup(LOG, writeBatch);
@@ -634,21 +638,23 @@ public class LeveldbApplicationTimelineStore extends AbstractService
    * write additional entries to the db for each primary filter.
    */
   private static void writePrimaryFilterEntries(WriteBatch writeBatch,
-      Map<String, Object> primaryFilters, byte[] key, byte[] value)
+      Map<String, Set<Object>> primaryFilters, byte[] key, byte[] value)
       throws IOException {
     if (primaryFilters != null && !primaryFilters.isEmpty()) {
-      for (Entry<String, Object> p : primaryFilters.entrySet()) {
-        writeBatch.put(addPrimaryFilterToKey(p.getKey(), p.getValue(),
-            key), value);
+      for (Entry<String, Set<Object>> pf : primaryFilters.entrySet()) {
+        for (Object pfval : pf.getValue()) {
+          writeBatch.put(addPrimaryFilterToKey(pf.getKey(), pfval,
+              key), value);
+        }
       }
     }
   }
 
   @Override
-  public ATSPutErrors put(ATSEntities atsEntities) {
-    ATSPutErrors response = new ATSPutErrors();
-    for (ATSEntity atsEntity : atsEntities.getEntities()) {
-      put(atsEntity, response);
+  public TimelinePutResponse put(TimelineEntities entities) {
+    TimelinePutResponse response = new TimelinePutResponse();
+    for (TimelineEntity entity : entities.getEntities()) {
+      put(entity, response);
     }
     return response;
   }
@@ -668,7 +674,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
    * @throws IOException
    */
   private byte[] getStartTime(String entityId, String entityType,
-      Long startTime, List<ATSEvent> events, WriteBatch writeBatch)
+      Long startTime, List<TimelineEvent> events, WriteBatch writeBatch)
       throws IOException {
     EntityIdentifier entity = new EntityIdentifier(entityId, entityType);
     if (startTime == null) {
@@ -688,7 +694,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
             return null;
           }
           Long min = Long.MAX_VALUE;
-          for (ATSEvent e : events)
+          for (TimelineEvent e : events)
             if (min > e.getTimestamp())
               min = e.getTimestamp();
           startTime = min;
@@ -764,13 +770,13 @@ public class LeveldbApplicationTimelineStore extends AbstractService
    * event type is not contained in the specified set of event types,
    * returns null.
    */
-  private static ATSEvent getEntityEvent(Set<String> eventTypes, byte[] key,
+  private static TimelineEvent getEntityEvent(Set<String> eventTypes, byte[] key,
       int offset, byte[] value) throws IOException {
     KeyParser kp = new KeyParser(key, offset);
     long ts = kp.getNextLong();
     String tstype = kp.getNextString();
     if (eventTypes == null || eventTypes.contains(tstype)) {
-      ATSEvent event = new ATSEvent();
+      TimelineEvent event = new TimelineEvent();
       event.setTimestamp(ts);
       event.setEventType(tstype);
       Object o = GenericObjectMapper.read(value);
@@ -790,13 +796,26 @@ public class LeveldbApplicationTimelineStore extends AbstractService
 
   /**
    * Creates a primary filter key, serializing ENTITY_ENTRY_PREFIX +
-   * entitytype + revstarttime + entity + PRIMARY_FILTER_COLUMN + name.
+   * entitytype + revstarttime + entity + PRIMARY_FILTER_COLUMN + name + value.
    */
   private static byte[] createPrimaryFilterKey(String entity,
-      String entitytype, byte[] revStartTime, String name) throws IOException {
+      String entitytype, byte[] revStartTime, String name, Object value)
+      throws IOException {
     return KeyBuilder.newInstance().add(ENTITY_ENTRY_PREFIX).add(entitytype)
         .add(revStartTime).add(entity).add(PRIMARY_FILTER_COLUMN).add(name)
-        .getBytes();
+        .add(GenericObjectMapper.write(value)).getBytes();
+  }
+
+  /**
+   * Parses the primary filter from the given key at the given offset and
+   * adds it to the given entity.
+   */
+  private static void addPrimaryFilter(TimelineEntity entity, byte[] key,
+      int offset) throws IOException {
+    KeyParser kp = new KeyParser(key, offset);
+    String name = kp.getNextString();
+    Object value = GenericObjectMapper.read(key, kp.getOffset());
+    entity.addPrimaryFilter(name, value);
   }
 
   /**
@@ -835,12 +854,12 @@ public class LeveldbApplicationTimelineStore extends AbstractService
    * Parses the related entity from the given key at the given offset and
    * adds it to the given entity.
    */
-  private static void addRelatedEntity(ATSEntity atsEntity, byte[] key,
+  private static void addRelatedEntity(TimelineEntity entity, byte[] key,
       int offset) throws IOException {
     KeyParser kp = new KeyParser(key, offset);
     String type = kp.getNextString();
     String id = kp.getNextString();
-    atsEntity.addRelatedEntity(type, id);
+    entity.addRelatedEntity(type, id);
   }
 
   /**

+ 54 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/MemoryApplicationTimelineStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/MemoryTimelineStore.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -33,16 +33,16 @@ import java.util.SortedSet;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents.ATSEventsOfOneEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 
 /**
- * In-memory implementation of {@link ApplicationTimelineStore}. This
+ * In-memory implementation of {@link TimelineStore}. This
  * implementation is for test purpose only. If users improperly instantiate it,
  * they may encounter reading and writing history data in different memory
  * store.
@@ -50,18 +50,18 @@ import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
  */
 @Private
 @Unstable
-public class MemoryApplicationTimelineStore
-    extends AbstractService implements ApplicationTimelineStore {
+public class MemoryTimelineStore
+    extends AbstractService implements TimelineStore {
 
-  private Map<EntityIdentifier, ATSEntity> entities =
-      new HashMap<EntityIdentifier, ATSEntity>();
+  private Map<EntityIdentifier, TimelineEntity> entities =
+      new HashMap<EntityIdentifier, TimelineEntity>();
 
-  public MemoryApplicationTimelineStore() {
-    super(MemoryApplicationTimelineStore.class.getName());
+  public MemoryTimelineStore() {
+    super(MemoryTimelineStore.class.getName());
   }
 
   @Override
-  public ATSEntities getEntities(String entityType, Long limit,
+  public TimelineEntities getEntities(String entityType, Long limit,
       Long windowStart, Long windowEnd, NameValuePair primaryFilter,
       Collection<NameValuePair> secondaryFilters, EnumSet<Field> fields) {
     if (limit == null) {
@@ -76,8 +76,8 @@ public class MemoryApplicationTimelineStore
     if (fields == null) {
       fields = EnumSet.allOf(Field.class);
     }
-    List<ATSEntity> entitiesSelected = new ArrayList<ATSEntity>();
-    for (ATSEntity entity : new PriorityQueue<ATSEntity>(entities.values())) {
+    List<TimelineEntity> entitiesSelected = new ArrayList<TimelineEntity>();
+    for (TimelineEntity entity : new PriorityQueue<TimelineEntity>(entities.values())) {
       if (entitiesSelected.size() >= limit) {
         break;
       }
@@ -91,7 +91,7 @@ public class MemoryApplicationTimelineStore
         continue;
       }
       if (primaryFilter != null &&
-          !matchFilter(entity.getPrimaryFilters(), primaryFilter)) {
+          !matchPrimaryFilter(entity.getPrimaryFilters(), primaryFilter)) {
         continue;
       }
       if (secondaryFilters != null) { // OR logic
@@ -109,23 +109,23 @@ public class MemoryApplicationTimelineStore
       }
       entitiesSelected.add(entity);
     }
-    List<ATSEntity> entitiesToReturn = new ArrayList<ATSEntity>();
-    for (ATSEntity entitySelected : entitiesSelected) {
+    List<TimelineEntity> entitiesToReturn = new ArrayList<TimelineEntity>();
+    for (TimelineEntity entitySelected : entitiesSelected) {
       entitiesToReturn.add(maskFields(entitySelected, fields));
     }
     Collections.sort(entitiesToReturn);
-    ATSEntities entitiesWrapper = new ATSEntities();
+    TimelineEntities entitiesWrapper = new TimelineEntities();
     entitiesWrapper.setEntities(entitiesToReturn);
     return entitiesWrapper;
   }
 
   @Override
-  public ATSEntity getEntity(String entityId, String entityType,
+  public TimelineEntity getEntity(String entityId, String entityType,
       EnumSet<Field> fieldsToRetrieve) {
     if (fieldsToRetrieve == null) {
       fieldsToRetrieve = EnumSet.allOf(Field.class);
     }
-    ATSEntity entity = entities.get(new EntityIdentifier(entityId, entityType));
+    TimelineEntity entity = entities.get(new EntityIdentifier(entityId, entityType));
     if (entity == null) {
       return null;
     } else {
@@ -134,11 +134,11 @@ public class MemoryApplicationTimelineStore
   }
 
   @Override
-  public ATSEvents getEntityTimelines(String entityType,
+  public TimelineEvents getEntityTimelines(String entityType,
       SortedSet<String> entityIds, Long limit, Long windowStart,
       Long windowEnd,
       Set<String> eventTypes) {
-    ATSEvents allEvents = new ATSEvents();
+    TimelineEvents allEvents = new TimelineEvents();
     if (entityIds == null) {
       return allEvents;
     }
@@ -153,14 +153,14 @@ public class MemoryApplicationTimelineStore
     }
     for (String entityId : entityIds) {
       EntityIdentifier entityID = new EntityIdentifier(entityId, entityType);
-      ATSEntity entity = entities.get(entityID);
+      TimelineEntity entity = entities.get(entityID);
       if (entity == null) {
         continue;
       }
-      ATSEventsOfOneEntity events = new ATSEventsOfOneEntity();
+      EventsOfOneEntity events = new EventsOfOneEntity();
       events.setEntityId(entityId);
       events.setEntityType(entityType);
-      for (ATSEvent event : entity.getEvents()) {
+      for (TimelineEvent event : entity.getEvents()) {
         if (events.getEvents().size() >= limit) {
           break;
         }
@@ -181,15 +181,15 @@ public class MemoryApplicationTimelineStore
   }
 
   @Override
-  public ATSPutErrors put(ATSEntities data) {
-    ATSPutErrors errors = new ATSPutErrors();
-    for (ATSEntity entity : data.getEntities()) {
+  public TimelinePutResponse put(TimelineEntities data) {
+    TimelinePutResponse response = new TimelinePutResponse();
+    for (TimelineEntity entity : data.getEntities()) {
       EntityIdentifier entityId =
           new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
       // store entity info in memory
-      ATSEntity existingEntity = entities.get(entityId);
+      TimelineEntity existingEntity = entities.get(entityId);
       if (existingEntity == null) {
-        existingEntity = new ATSEntity();
+        existingEntity = new TimelineEntity();
         existingEntity.setEntityId(entity.getEntityId());
         existingEntity.setEntityType(entity.getEntityType());
         existingEntity.setStartTime(entity.getStartTime());
@@ -207,11 +207,11 @@ public class MemoryApplicationTimelineStore
       if (existingEntity.getStartTime() == null) {
         if (existingEntity.getEvents() == null
             || existingEntity.getEvents().isEmpty()) {
-          ATSPutError error = new ATSPutError();
+          TimelinePutError error = new TimelinePutError();
           error.setEntityId(entityId.getId());
           error.setEntityType(entityId.getType());
-          error.setErrorCode(ATSPutError.NO_START_TIME);
-          errors.addError(error);
+          error.setErrorCode(TimelinePutError.NO_START_TIME);
+          response.addError(error);
           entities.remove(entityId);
           continue;
         } else {
@@ -236,7 +236,7 @@ public class MemoryApplicationTimelineStore
       if (entity.getRelatedEntities() == null) {
         continue;
       }
-      for (Map.Entry<String, List<String>> partRelatedEntities : entity
+      for (Map.Entry<String, Set<String>> partRelatedEntities : entity
           .getRelatedEntities().entrySet()) {
         if (partRelatedEntities == null) {
           continue;
@@ -244,12 +244,12 @@ public class MemoryApplicationTimelineStore
         for (String idStr : partRelatedEntities.getValue()) {
           EntityIdentifier relatedEntityId =
               new EntityIdentifier(idStr, partRelatedEntities.getKey());
-          ATSEntity relatedEntity = entities.get(relatedEntityId);
+          TimelineEntity relatedEntity = entities.get(relatedEntityId);
           if (relatedEntity != null) {
             relatedEntity.addRelatedEntity(
                 existingEntity.getEntityType(), existingEntity.getEntityId());
           } else {
-            relatedEntity = new ATSEntity();
+            relatedEntity = new TimelineEntity();
             relatedEntity.setEntityId(relatedEntityId.getId());
             relatedEntity.setEntityType(relatedEntityId.getType());
             relatedEntity.setStartTime(existingEntity.getStartTime());
@@ -260,13 +260,13 @@ public class MemoryApplicationTimelineStore
         }
       }
     }
-    return errors;
+    return response;
   }
 
-  private static ATSEntity maskFields(
-      ATSEntity entity, EnumSet<Field> fields) {
+  private static TimelineEntity maskFields(
+      TimelineEntity entity, EnumSet<Field> fields) {
     // Conceal the fields that are not going to be exposed
-    ATSEntity entityToReturn = new ATSEntity();
+    TimelineEntity entityToReturn = new TimelineEntity();
     entityToReturn.setEntityId(entity.getEntityId());
     entityToReturn.setEntityType(entity.getEntityType());
     entityToReturn.setStartTime(entity.getStartTime());
@@ -293,4 +293,14 @@ public class MemoryApplicationTimelineStore
     return true;
   }
 
+  private static boolean matchPrimaryFilter(Map<String, Set<Object>> tags,
+      NameValuePair filter) {
+    Set<Object> value = tags.get(filter.getName());
+    if (value == null) { // doesn't have the filter
+      return false;
+    } else {
+      return value.contains(filter.getValue());
+    }
+  }
+
 }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/NameValuePair.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/NameValuePair.java

@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
  * A class holding a name and value pair, used for specifying filters in
- * {@link ApplicationTimelineReader}.
+ * {@link TimelineReader}.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

+ 141 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineReader.java

@@ -0,0 +1,141 @@
+/**
+ * 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.applicationhistoryservice.timeline;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+
+/**
+ * This interface is for retrieving timeline information.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface TimelineReader {
+
+  /**
+   * Possible fields to retrieve for {@link #getEntities} and {@link #getEntity}
+   * .
+   */
+  enum Field {
+    EVENTS,
+    RELATED_ENTITIES,
+    PRIMARY_FILTERS,
+    OTHER_INFO,
+    LAST_EVENT_ONLY
+  }
+
+  /**
+   * Default limit for {@link #getEntities} and {@link #getEntityTimelines}.
+   */
+  final long DEFAULT_LIMIT = 100;
+
+  /**
+   * This method retrieves a list of entity information, {@link TimelineEntity}, sorted
+   * by the starting timestamp for the entity, descending.
+   * 
+   * @param entityType
+   *          The type of entities to return (required).
+   * @param limit
+   *          A limit on the number of entities to return. If null, defaults to
+   *          {@link #DEFAULT_LIMIT}.
+   * @param windowStart
+   *          The earliest start timestamp to retrieve (exclusive). If null,
+   *          defaults to retrieving all entities until the limit is reached.
+   * @param windowEnd
+   *          The latest start timestamp to retrieve (inclusive). If null,
+   *          defaults to {@link Long#MAX_VALUE}
+   * @param primaryFilter
+   *          Retrieves only entities that have the specified primary filter. If
+   *          null, retrieves all entities. This is an indexed retrieval, and no
+   *          entities that do not match the filter are scanned.
+   * @param secondaryFilters
+   *          Retrieves only entities that have exact matches for all the
+   *          specified filters in their primary filters or other info. This is
+   *          not an indexed retrieval, so all entities are scanned but only
+   *          those matching the filters are returned.
+   * @param fieldsToRetrieve
+   *          Specifies which fields of the entity object to retrieve (see
+   *          {@link Field}). If the set of fields contains
+   *          {@link Field#LAST_EVENT_ONLY} and not {@link Field#EVENTS}, the
+   *          most recent event for each entity is retrieved. If null, retrieves
+   *          all fields.
+   * @return An {@link TimelineEntities} object.
+   * @throws IOException
+   */
+  TimelineEntities getEntities(String entityType,
+      Long limit, Long windowStart, Long windowEnd,
+      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
+      EnumSet<Field> fieldsToRetrieve) throws IOException;
+
+  /**
+   * This method retrieves the entity information for a given entity.
+   * 
+   * @param entityId
+   *          The entity whose information will be retrieved.
+   * @param entityType
+   *          The type of the entity.
+   * @param fieldsToRetrieve
+   *          Specifies which fields of the entity object to retrieve (see
+   *          {@link Field}). If the set of fields contains
+   *          {@link Field#LAST_EVENT_ONLY} and not {@link Field#EVENTS}, the
+   *          most recent event for each entity is retrieved. If null, retrieves
+   *          all fields.
+   * @return An {@link TimelineEntity} object.
+   * @throws IOException
+   */
+  TimelineEntity getEntity(String entityId, String entityType, EnumSet<Field>
+      fieldsToRetrieve) throws IOException;
+
+  /**
+   * This method retrieves the events for a list of entities all of the same
+   * entity type. The events for each entity are sorted in order of their
+   * timestamps, descending.
+   * 
+   * @param entityType
+   *          The type of entities to retrieve events for.
+   * @param entityIds
+   *          The entity IDs to retrieve events for.
+   * @param limit
+   *          A limit on the number of events to return for each entity. If
+   *          null, defaults to {@link #DEFAULT_LIMIT} events per entity.
+   * @param windowStart
+   *          If not null, retrieves only events later than the given time
+   *          (exclusive)
+   * @param windowEnd
+   *          If not null, retrieves only events earlier than the given time
+   *          (inclusive)
+   * @param eventTypes
+   *          Restricts the events returned to the given types. If null, events
+   *          of all types will be returned.
+   * @return An {@link TimelineEvents} object.
+   * @throws IOException
+   */
+  TimelineEvents getEntityTimelines(String entityType,
+      SortedSet<String> entityIds, Long limit, Long windowStart,
+      Long windowEnd, Set<String> eventTypes) throws IOException;
+}

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineStore.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -24,6 +24,6 @@ import org.apache.hadoop.service.Service;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public interface ApplicationTimelineStore extends
-    Service, ApplicationTimelineReader, ApplicationTimelineWriter {
+public interface TimelineStore extends
+    Service, TimelineReader, TimelineWriter {
 }

+ 12 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineWriter.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineWriter.java

@@ -16,31 +16,31 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 
 import java.io.IOException;
 
 /**
- * This interface is for storing application timeline information.
+ * This interface is for storing timeline information.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public interface ApplicationTimelineWriter {
+public interface TimelineWriter {
 
   /**
-   * Stores entity information to the application timeline store. Any errors
-   * occurring for individual put request objects will be reported in the
-   * response.
-   *
-   * @param data An {@link ATSEntities} object.
-   * @return An {@link ATSPutErrors} object.
+   * Stores entity information to the timeline store. Any errors occurring for
+   * individual put request objects will be reported in the response.
+   * 
+   * @param data
+   *          An {@link TimelineEntities} object.
+   * @return An {@link TimelinePutResponse} object.
    * @throws IOException
    */
-  ATSPutErrors put(ATSEntities data) throws IOException;
+  TimelinePutResponse put(TimelineEntities data) throws IOException;
 
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/package-info.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/package-info.java

@@ -16,5 +16,5 @@
  * limitations under the License.
  */
 @InterfaceAudience.Private
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 import org.apache.hadoop.classification.InterfaceAudience;

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java

@@ -21,7 +21,7 @@ import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
 
 import org.apache.hadoop.yarn.server.api.ApplicationContext;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
@@ -30,22 +30,22 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class AHSWebApp extends WebApp implements YarnWebParams {
 
   private final ApplicationHistoryManager applicationHistoryManager;
-  private final ApplicationTimelineStore applicationTimelineStore;
+  private final TimelineStore timelineStore;
 
   public AHSWebApp(ApplicationHistoryManager applicationHistoryManager,
-      ApplicationTimelineStore applicationTimelineStore) {
+      TimelineStore timelineStore) {
     this.applicationHistoryManager = applicationHistoryManager;
-    this.applicationTimelineStore = applicationTimelineStore;
+    this.timelineStore = timelineStore;
   }
 
   @Override
   public void setup() {
     bind(YarnJacksonJaxbJsonProvider.class);
     bind(AHSWebServices.class);
-    bind(ATSWebServices.class);
+    bind(TimelineWebServices.class);
     bind(GenericExceptionHandler.class);
     bind(ApplicationContext.class).toInstance(applicationHistoryManager);
-    bind(ApplicationTimelineStore.class).toInstance(applicationTimelineStore);
+    bind(TimelineStore.class).toInstance(timelineStore);
     route("/", AHSController.class);
     route(pajoin("/apps", APP_STATE), AHSController.class);
     route(pajoin("/app", APPLICATION_ID), AHSController.class, "app");

+ 25 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ATSWebServices.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TimelineWebServices.java

@@ -50,29 +50,29 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineReader.Field;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.NameValuePair;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineReader.Field;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 
 @Singleton
-@Path("/ws/v1/apptimeline")
+@Path("/ws/v1/timeline")
 //TODO: support XML serialization/deserialization
-public class ATSWebServices {
+public class TimelineWebServices {
 
-  private static final Log LOG = LogFactory.getLog(ATSWebServices.class);
+  private static final Log LOG = LogFactory.getLog(TimelineWebServices.class);
 
-  private ApplicationTimelineStore store;
+  private TimelineStore store;
 
   @Inject
-  public ATSWebServices(ApplicationTimelineStore store) {
+  public TimelineWebServices(TimelineStore store) {
     this.store = store;
   }
 
@@ -104,7 +104,7 @@ public class ATSWebServices {
   }
 
   /**
-   * Return the description of the application timeline web services.
+   * Return the description of the timeline web services.
    */
   @GET
   @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
@@ -112,7 +112,7 @@ public class ATSWebServices {
       @Context HttpServletRequest req,
       @Context HttpServletResponse res) {
     init(res);
-    return new AboutInfo("Application Timeline API");
+    return new AboutInfo("Timeline API");
   }
 
   /**
@@ -121,7 +121,7 @@ public class ATSWebServices {
   @GET
   @Path("/{entityType}")
   @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
-  public ATSEntities getEntities(
+  public TimelineEntities getEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("entityType") String entityType,
@@ -132,7 +132,7 @@ public class ATSWebServices {
       @QueryParam("limit") String limit,
       @QueryParam("fields") String fields) {
     init(res);
-    ATSEntities entities = null;
+    TimelineEntities entities = null;
     try {
       entities = store.getEntities(
           parseStr(entityType),
@@ -153,7 +153,7 @@ public class ATSWebServices {
           Response.Status.INTERNAL_SERVER_ERROR);
     }
     if (entities == null) {
-      return new ATSEntities();
+      return new TimelineEntities();
     }
     return entities;
   }
@@ -164,14 +164,14 @@ public class ATSWebServices {
   @GET
   @Path("/{entityType}/{entityId}")
   @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
-  public ATSEntity getEntity(
+  public TimelineEntity getEntity(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("entityType") String entityType,
       @PathParam("entityId") String entityId,
       @QueryParam("fields") String fields) {
     init(res);
-    ATSEntity entity = null;
+    TimelineEntity entity = null;
     try {
       entity =
           store.getEntity(parseStr(entityId), parseStr(entityType),
@@ -196,7 +196,7 @@ public class ATSWebServices {
   @GET
   @Path("/{entityType}/events")
   @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
-  public ATSEvents getEvents(
+  public TimelineEvents getEvents(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("entityType") String entityType,
@@ -206,7 +206,7 @@ public class ATSWebServices {
       @QueryParam("windowEnd") String windowEnd,
       @QueryParam("limit") String limit) {
     init(res);
-    ATSEvents events = null;
+    TimelineEvents events = null;
     try {
       events = store.getEntityTimelines(
           parseStr(entityType),
@@ -224,7 +224,7 @@ public class ATSWebServices {
           Response.Status.INTERNAL_SERVER_ERROR);
     }
     if (events == null) {
-      return new ATSEvents();
+      return new TimelineEvents();
     }
     return events;
   }
@@ -235,13 +235,13 @@ public class ATSWebServices {
    */
   @POST
   @Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
-  public ATSPutErrors postEntities(
+  public TimelinePutResponse postEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
-      ATSEntities entities) {
+      TimelineEntities entities) {
     init(res);
     if (entities == null) {
-      return new ATSPutErrors();
+      return new TimelinePutResponse();
     }
     try {
       return store.put(entities);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java

@@ -57,7 +57,7 @@ public class TestApplicationHistoryClientService extends
   public void setup() {
     historyServer = new ApplicationHistoryServer();
     Configuration config = new YarnConfiguration();
-    config.setClass(YarnConfiguration.AHS_STORE,
+    config.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
       MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
     historyServer.init(config);
     historyServer.start();

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerImpl.java

@@ -38,7 +38,7 @@ public class TestApplicationHistoryManagerImpl extends
   @Before
   public void setup() throws Exception {
     Configuration config = new Configuration();
-    config.setClass(YarnConfiguration.AHS_STORE,
+    config.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
       MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
     applicationHistoryManagerImpl = new ApplicationHistoryManagerImpl();
     applicationHistoryManagerImpl.init(config);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java

@@ -52,7 +52,7 @@ public class TestFileSystemApplicationHistoryStore extends
     fs.initialize(new URI("/"), conf);
     fsWorkingPath = new Path("Test");
     fs.delete(fsWorkingPath, true);
-    conf.set(YarnConfiguration.FS_HISTORY_STORE_URI, fsWorkingPath.toString());
+    conf.set(YarnConfiguration.FS_APPLICATION_HISTORY_STORE_URI, fsWorkingPath.toString());
     store = new FileSystemApplicationHistoryStore();
     store.init(conf);
     store.start();

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/TestGenericObjectMapper.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TestGenericObjectMapper.java

@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper;
 import org.junit.Test;
 
 import java.io.IOException;

+ 6 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/TestLeveldbApplicationTimelineStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TestLeveldbTimelineStore.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import java.io.File;
 import java.io.IOException;
@@ -25,21 +25,15 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class TestLeveldbApplicationTimelineStore
-    extends ApplicationTimelineStoreTestUtils {
+public class TestLeveldbTimelineStore
+    extends TimelineStoreTestUtils {
   private FileContext fsContext;
   private File fsPath;
 
@@ -50,9 +44,9 @@ public class TestLeveldbApplicationTimelineStore
     fsPath = new File("target", this.getClass().getSimpleName() +
         "-tmpDir").getAbsoluteFile();
     fsContext.delete(new Path(fsPath.getAbsolutePath()), true);
-    conf.set(YarnConfiguration.ATS_LEVELDB_PATH_PROPERTY,
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH,
         fsPath.getAbsolutePath());
-    store = new LeveldbApplicationTimelineStore();
+    store = new LeveldbTimelineStore();
     store.init(conf);
     store.start();
     loadTestData();
@@ -68,7 +62,7 @@ public class TestLeveldbApplicationTimelineStore
   @Test
   public void testGetSingleEntity() throws IOException {
     super.testGetSingleEntity();
-    ((LeveldbApplicationTimelineStore)store).clearStartTimeCache();
+    ((LeveldbTimelineStore)store).clearStartTimeCache();
     super.testGetSingleEntity();
   }
 

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/TestMemoryApplicationTimelineStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TestMemoryTimelineStore.java

@@ -16,21 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.MemoryTimelineStore;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
 
-public class TestMemoryApplicationTimelineStore
-    extends ApplicationTimelineStoreTestUtils {
+public class TestMemoryTimelineStore
+    extends TimelineStoreTestUtils {
 
   @Before
   public void setup() throws Exception {
-    store = new MemoryApplicationTimelineStore();
+    store = new MemoryTimelineStore();
     store.init(new YarnConfiguration());
     store.start();
     loadTestData();
@@ -42,7 +44,7 @@ public class TestMemoryApplicationTimelineStore
     store.stop();
   }
 
-  public ApplicationTimelineStore getApplicationTimelineStore() {
+  public TimelineStore getTimelineStore() {
     return store;
   }
 

+ 204 - 177
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/apptimeline/ApplicationTimelineStoreTestUtils.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/timeline/TimelineStoreTestUtils.java

@@ -15,69 +15,80 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
+package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents.ATSEventsOfOneEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineReader.Field;
-
-public class ApplicationTimelineStoreTestUtils {
-
-  private static final Map<String, Object> EMPTY_MAP = Collections.emptyMap();
-  private static final Map<String, List<String>> EMPTY_REL_ENTITIES =
-      new HashMap<String, List<String>>();
-
-  protected ApplicationTimelineStore store;
-  private String entity1;
-  private String entityType1;
-  private String entity1b;
-  private String entity2;
-  private String entityType2;
-  private Map<String, Object> primaryFilters;
-  private Map<String, Object> secondaryFilters;
-  private Map<String, Object> allFilters;
-  private Map<String, Object> otherInfo;
-  private Map<String, List<String>> relEntityMap;
-  private NameValuePair userFilter;
-  private Collection<NameValuePair> goodTestingFilters;
-  private Collection<NameValuePair> badTestingFilters;
-  private ATSEvent ev1;
-  private ATSEvent ev2;
-  private ATSEvent ev3;
-  private ATSEvent ev4;
-  private Map<String, Object> eventInfo;
-  private List<ATSEvent> events1;
-  private List<ATSEvent> events2;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineReader.Field;
+
+public class TimelineStoreTestUtils {
+
+  protected static final Map<String, Object> EMPTY_MAP =
+      Collections.emptyMap();
+  protected static final Map<String, Set<Object>> EMPTY_PRIMARY_FILTERS =
+      Collections.emptyMap();
+  protected static final Map<String, Set<String>> EMPTY_REL_ENTITIES =
+      Collections.emptyMap();
+
+  protected TimelineStore store;
+  protected String entityId1;
+  protected String entityType1;
+  protected String entityId1b;
+  protected String entityId2;
+  protected String entityType2;
+  protected Map<String, Set<Object>> primaryFilters;
+  protected Map<String, Object> secondaryFilters;
+  protected Map<String, Object> allFilters;
+  protected Map<String, Object> otherInfo;
+  protected Map<String, Set<String>> relEntityMap;
+  protected NameValuePair userFilter;
+  protected Collection<NameValuePair> goodTestingFilters;
+  protected Collection<NameValuePair> badTestingFilters;
+  protected TimelineEvent ev1;
+  protected TimelineEvent ev2;
+  protected TimelineEvent ev3;
+  protected TimelineEvent ev4;
+  protected Map<String, Object> eventInfo;
+  protected List<TimelineEvent> events1;
+  protected List<TimelineEvent> events2;
 
   /**
    * Load test data into the given store
    */
   protected void loadTestData() throws IOException {
-    ATSEntities atsEntities = new ATSEntities();
-    Map<String, Object> primaryFilters = new HashMap<String, Object>();
-    primaryFilters.put("user", "username");
-    primaryFilters.put("appname", 12345l);
+    TimelineEntities entities = new TimelineEntities();
+    Map<String, Set<Object>> primaryFilters =
+        new HashMap<String, Set<Object>>();
+    Set<Object> l1 = new HashSet<Object>();
+    l1.add("username");
+    Set<Object> l2 = new HashSet<Object>();
+    l2.add(12345l);
+    primaryFilters.put("user", l1);
+    primaryFilters.put("appname", l2);
     Map<String, Object> secondaryFilters = new HashMap<String, Object>();
     secondaryFilters.put("startTime", 123456l);
     secondaryFilters.put("status", "RUNNING");
@@ -85,62 +96,62 @@ public class ApplicationTimelineStoreTestUtils {
     otherInfo1.put("info1", "val1");
     otherInfo1.putAll(secondaryFilters);
 
-    String entity1 = "id_1";
+    String entityId1 = "id_1";
     String entityType1 = "type_1";
-    String entity1b = "id_2";
-    String entity2 = "id_2";
+    String entityId1b = "id_2";
+    String entityId2 = "id_2";
     String entityType2 = "type_2";
 
-    Map<String, List<String>> relatedEntities =
-        new HashMap<String, List<String>>();
-    relatedEntities.put(entityType2, Collections.singletonList(entity2));
+    Map<String, Set<String>> relatedEntities =
+        new HashMap<String, Set<String>>();
+    relatedEntities.put(entityType2, Collections.singleton(entityId2));
 
-    ATSEvent ev3 = createEvent(789l, "launch_event", null);
-    ATSEvent ev4 = createEvent(-123l, "init_event", null);
-    List<ATSEvent> events = new ArrayList<ATSEvent>();
+    TimelineEvent ev3 = createEvent(789l, "launch_event", null);
+    TimelineEvent ev4 = createEvent(-123l, "init_event", null);
+    List<TimelineEvent> events = new ArrayList<TimelineEvent>();
     events.add(ev3);
     events.add(ev4);
-    atsEntities.setEntities(Collections.singletonList(createEntity(entity2,
+    entities.setEntities(Collections.singletonList(createEntity(entityId2,
         entityType2, null, events, null, null, null)));
-    ATSPutErrors response = store.put(atsEntities);
+    TimelinePutResponse response = store.put(entities);
     assertEquals(0, response.getErrors().size());
 
-    ATSEvent ev1 = createEvent(123l, "start_event", null);
-    atsEntities.setEntities(Collections.singletonList(createEntity(entity1,
+    TimelineEvent ev1 = createEvent(123l, "start_event", null);
+    entities.setEntities(Collections.singletonList(createEntity(entityId1,
         entityType1, 123l, Collections.singletonList(ev1),
         relatedEntities, primaryFilters, otherInfo1)));
-    response = store.put(atsEntities);
+    response = store.put(entities);
     assertEquals(0, response.getErrors().size());
-    atsEntities.setEntities(Collections.singletonList(createEntity(entity1b,
+    entities.setEntities(Collections.singletonList(createEntity(entityId1b,
         entityType1, null, Collections.singletonList(ev1), relatedEntities,
         primaryFilters, otherInfo1)));
-    response = store.put(atsEntities);
+    response = store.put(entities);
     assertEquals(0, response.getErrors().size());
 
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put("event info 1", "val1");
-    ATSEvent ev2 = createEvent(456l, "end_event", eventInfo);
+    TimelineEvent ev2 = createEvent(456l, "end_event", eventInfo);
     Map<String, Object> otherInfo2 = new HashMap<String, Object>();
     otherInfo2.put("info2", "val2");
-    atsEntities.setEntities(Collections.singletonList(createEntity(entity1,
+    entities.setEntities(Collections.singletonList(createEntity(entityId1,
         entityType1, null, Collections.singletonList(ev2), null,
         primaryFilters, otherInfo2)));
-    response = store.put(atsEntities);
+    response = store.put(entities);
     assertEquals(0, response.getErrors().size());
-    atsEntities.setEntities(Collections.singletonList(createEntity(entity1b,
+    entities.setEntities(Collections.singletonList(createEntity(entityId1b,
         entityType1, 789l, Collections.singletonList(ev2), null,
         primaryFilters, otherInfo2)));
-    response = store.put(atsEntities);
+    response = store.put(entities);
     assertEquals(0, response.getErrors().size());
 
-    atsEntities.setEntities(Collections.singletonList(createEntity(
+    entities.setEntities(Collections.singletonList(createEntity(
         "badentityid", "badentity", null, null, null, null, otherInfo1)));
-    response = store.put(atsEntities);
+    response = store.put(entities);
     assertEquals(1, response.getErrors().size());
-    ATSPutError error = response.getErrors().get(0);
+    TimelinePutError error = response.getErrors().get(0);
     assertEquals("badentityid", error.getEntityId());
     assertEquals("badentity", error.getEntityType());
-    assertEquals(ATSPutError.NO_START_TIME, error.getErrorCode());
+    assertEquals(TimelinePutError.NO_START_TIME, error.getErrorCode());
   }
 
   /**
@@ -156,24 +167,32 @@ public class ApplicationTimelineStoreTestUtils {
     badTestingFilters.add(new NameValuePair("appname", 12345l));
     badTestingFilters.add(new NameValuePair("status", "FINISHED"));
 
-    primaryFilters = new HashMap<String, Object>();
-    primaryFilters.put("user", "username");
-    primaryFilters.put("appname", 12345l);
+    primaryFilters = new HashMap<String, Set<Object>>();
+    Set<Object> l1 = new HashSet<Object>();
+    l1.add("username");
+    Set<Object> l2 = new HashSet<Object>();
+    l2.add(12345l);
+    primaryFilters.put("user", l1);
+    primaryFilters.put("appname", l2);
     secondaryFilters = new HashMap<String, Object>();
     secondaryFilters.put("startTime", 123456l);
     secondaryFilters.put("status", "RUNNING");
     allFilters = new HashMap<String, Object>();
     allFilters.putAll(secondaryFilters);
-    allFilters.putAll(primaryFilters);
+    for (Entry<String, Set<Object>> pf : primaryFilters.entrySet()) {
+      for (Object o : pf.getValue()) {
+        allFilters.put(pf.getKey(), o);
+      }
+    }
     otherInfo = new HashMap<String, Object>();
     otherInfo.put("info1", "val1");
     otherInfo.put("info2", "val2");
     otherInfo.putAll(secondaryFilters);
 
-    entity1 = "id_1";
+    entityId1 = "id_1";
     entityType1 = "type_1";
-    entity1b = "id_2";
-    entity2 = "id_2";
+    entityId1b = "id_2";
+    entityId2 = "id_2";
     entityType2 = "type_2";
 
     ev1 = createEvent(123l, "start_event", null);
@@ -181,20 +200,20 @@ public class ApplicationTimelineStoreTestUtils {
     eventInfo = new HashMap<String, Object>();
     eventInfo.put("event info 1", "val1");
     ev2 = createEvent(456l, "end_event", eventInfo);
-    events1 = new ArrayList<ATSEvent>();
+    events1 = new ArrayList<TimelineEvent>();
     events1.add(ev2);
     events1.add(ev1);
 
     relEntityMap =
-        new HashMap<String, List<String>>();
-    List<String> ids = new ArrayList<String>();
-    ids.add(entity1);
-    ids.add(entity1b);
+        new HashMap<String, Set<String>>();
+    Set<String> ids = new HashSet<String>();
+    ids.add(entityId1);
+    ids.add(entityId1b);
     relEntityMap.put(entityType1, ids);
 
     ev3 = createEvent(789l, "launch_event", null);
     ev4 = createEvent(-123l, "init_event", null);
-    events2 = new ArrayList<ATSEvent>();
+    events2 = new ArrayList<TimelineEvent>();
     events2.add(ev3);
     events2.add(ev4);
   }
@@ -204,39 +223,39 @@ public class ApplicationTimelineStoreTestUtils {
     verifyEntityInfo(null, null, null, null, null, null,
         store.getEntity("id_1", "type_2", EnumSet.allOf(Field.class)));
 
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, store.getEntity(entity1, entityType1,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
+        primaryFilters, otherInfo, store.getEntity(entityId1, entityType1,
         EnumSet.allOf(Field.class)));
 
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, store.getEntity(entity1b, entityType1,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
+        primaryFilters, otherInfo, store.getEntity(entityId1b, entityType1,
         EnumSet.allOf(Field.class)));
 
-    verifyEntityInfo(entity2, entityType2, events2, relEntityMap, EMPTY_MAP,
-        EMPTY_MAP, store.getEntity(entity2, entityType2,
+    verifyEntityInfo(entityId2, entityType2, events2, relEntityMap,
+        EMPTY_PRIMARY_FILTERS, EMPTY_MAP, store.getEntity(entityId2, entityType2,
         EnumSet.allOf(Field.class)));
 
     // test getting single fields
-    verifyEntityInfo(entity1, entityType1, events1, null, null, null,
-        store.getEntity(entity1, entityType1, EnumSet.of(Field.EVENTS)));
+    verifyEntityInfo(entityId1, entityType1, events1, null, null, null,
+        store.getEntity(entityId1, entityType1, EnumSet.of(Field.EVENTS)));
 
-    verifyEntityInfo(entity1, entityType1, Collections.singletonList(ev2),
-        null, null, null, store.getEntity(entity1, entityType1,
+    verifyEntityInfo(entityId1, entityType1, Collections.singletonList(ev2),
+        null, null, null, store.getEntity(entityId1, entityType1,
         EnumSet.of(Field.LAST_EVENT_ONLY)));
 
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, store.getEntity(entity1b, entityType1,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
+        primaryFilters, otherInfo, store.getEntity(entityId1b, entityType1,
         null));
 
-    verifyEntityInfo(entity1, entityType1, null, null, primaryFilters, null,
-        store.getEntity(entity1, entityType1,
+    verifyEntityInfo(entityId1, entityType1, null, null, primaryFilters, null,
+        store.getEntity(entityId1, entityType1,
             EnumSet.of(Field.PRIMARY_FILTERS)));
 
-    verifyEntityInfo(entity1, entityType1, null, null, null, otherInfo,
-        store.getEntity(entity1, entityType1, EnumSet.of(Field.OTHER_INFO)));
+    verifyEntityInfo(entityId1, entityType1, null, null, null, otherInfo,
+        store.getEntity(entityId1, entityType1, EnumSet.of(Field.OTHER_INFO)));
 
-    verifyEntityInfo(entity2, entityType2, null, relEntityMap, null, null,
-        store.getEntity(entity2, entityType2,
+    verifyEntityInfo(entityId2, entityType2, null, relEntityMap, null, null,
+        store.getEntity(entityId2, entityType2,
             EnumSet.of(Field.RELATED_ENTITIES)));
   }
 
@@ -255,31 +274,31 @@ public class ApplicationTimelineStoreTestUtils {
         store.getEntities("type_3", null, null, null, userFilter,
             null, null).getEntities().size());
 
-    List<ATSEntity> entities =
+    List<TimelineEntity> entities =
         store.getEntities("type_1", null, null, null, null, null,
             EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
 
     entities = store.getEntities("type_2", null, null, null, null, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(1, entities.size());
-    verifyEntityInfo(entity2, entityType2, events2, relEntityMap, EMPTY_MAP,
-        EMPTY_MAP, entities.get(0));
+    verifyEntityInfo(entityId2, entityType2, events2, relEntityMap,
+        EMPTY_PRIMARY_FILTERS, EMPTY_MAP, entities.get(0));
 
     entities = store.getEntities("type_1", 1l, null, null, null, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(1, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
 
     entities = store.getEntities("type_1", 1l, 0l, null, null, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(1, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
 
     entities = store.getEntities("type_1", null, 234l, null, null, null,
@@ -297,17 +316,17 @@ public class ApplicationTimelineStoreTestUtils {
     entities = store.getEntities("type_1", null, null, 345l, null, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
 
     entities = store.getEntities("type_1", null, null, 123l, null, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
   }
 
@@ -326,12 +345,12 @@ public class ApplicationTimelineStoreTestUtils {
             new NameValuePair("none", "none"), null,
             EnumSet.allOf(Field.class)).getEntities().size());
 
-    List<ATSEntity> entities = store.getEntities("type_1", null, null, null,
+    List<TimelineEntity> entities = store.getEntities("type_1", null, null, null,
         userFilter, null, EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
 
     entities = store.getEntities("type_2", null, null, null, userFilter, null,
@@ -341,13 +360,13 @@ public class ApplicationTimelineStoreTestUtils {
     entities = store.getEntities("type_1", 1l, null, null, userFilter, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(1, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
 
     entities = store.getEntities("type_1", 1l, 0l, null, userFilter, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(1, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
 
     entities = store.getEntities("type_1", null, 234l, null, userFilter, null,
@@ -361,28 +380,28 @@ public class ApplicationTimelineStoreTestUtils {
     entities = store.getEntities("type_1", null, null, 345l, userFilter, null,
         EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
   }
 
   public void testGetEntitiesWithSecondaryFilters() throws IOException {
     // test using secondary filter
-    List<ATSEntity> entities = store.getEntities("type_1", null, null, null,
+    List<TimelineEntity> entities = store.getEntities("type_1", null, null, null,
         null, goodTestingFilters, EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
 
     entities = store.getEntities("type_1", null, null, null, userFilter,
         goodTestingFilters, EnumSet.allOf(Field.class)).getEntities();
     assertEquals(2, entities.size());
-    verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(0));
-    verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
+    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
         primaryFilters, otherInfo, entities.get(1));
 
     entities = store.getEntities("type_1", null, null, null, null,
@@ -397,102 +416,106 @@ public class ApplicationTimelineStoreTestUtils {
   public void testGetEvents() throws IOException {
     // test getting entity timelines
     SortedSet<String> sortedSet = new TreeSet<String>();
-    sortedSet.add(entity1);
-    List<ATSEventsOfOneEntity> timelines =
+    sortedSet.add(entityId1);
+    List<EventsOfOneEntity> timelines =
         store.getEntityTimelines(entityType1, sortedSet, null, null,
             null, null).getAllEvents();
     assertEquals(1, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2, ev1);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2, ev1);
 
-    sortedSet.add(entity1b);
+    sortedSet.add(entityId1b);
     timelines = store.getEntityTimelines(entityType1, sortedSet, null,
         null, null, null).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2, ev1);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2, ev1);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2, ev1);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2, ev1);
 
     timelines = store.getEntityTimelines(entityType1, sortedSet, 1l,
         null, null, null).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
 
     timelines = store.getEntityTimelines(entityType1, sortedSet, null,
         345l, null, null).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
 
     timelines = store.getEntityTimelines(entityType1, sortedSet, null,
         123l, null, null).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
 
     timelines = store.getEntityTimelines(entityType1, sortedSet, null,
         null, 345l, null).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev1);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev1);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev1);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev1);
 
     timelines = store.getEntityTimelines(entityType1, sortedSet, null,
         null, 123l, null).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev1);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev1);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev1);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev1);
 
     timelines = store.getEntityTimelines(entityType1, sortedSet, null,
         null, null, Collections.singleton("end_event")).getAllEvents();
     assertEquals(2, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
-    verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
+    verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
 
-    sortedSet.add(entity2);
+    sortedSet.add(entityId2);
     timelines = store.getEntityTimelines(entityType2, sortedSet, null,
         null, null, null).getAllEvents();
     assertEquals(1, timelines.size());
-    verifyEntityTimeline(timelines.get(0), entity2, entityType2, ev3, ev4);
+    verifyEntityTimeline(timelines.get(0), entityId2, entityType2, ev3, ev4);
   }
 
   /**
    * Verify a single entity
    */
-  private static void verifyEntityInfo(String entity, String entityType,
-      List<ATSEvent> events, Map<String, List<String>> relatedEntities,
-      Map<String, Object> primaryFilters, Map<String, Object> otherInfo,
-      ATSEntity retrievedEntityInfo) {
-    if (entity == null) {
+  protected static void verifyEntityInfo(String entityId, String entityType,
+      List<TimelineEvent> events, Map<String, Set<String>> relatedEntities,
+      Map<String, Set<Object>> primaryFilters, Map<String, Object> otherInfo,
+      TimelineEntity retrievedEntityInfo) {
+    if (entityId == null) {
       assertNull(retrievedEntityInfo);
       return;
     }
-    assertEquals(entity, retrievedEntityInfo.getEntityId());
+    assertEquals(entityId, retrievedEntityInfo.getEntityId());
     assertEquals(entityType, retrievedEntityInfo.getEntityType());
-    if (events == null)
+    if (events == null) {
       assertNull(retrievedEntityInfo.getEvents());
-    else
+    } else {
       assertEquals(events, retrievedEntityInfo.getEvents());
-    if (relatedEntities == null)
+    }
+    if (relatedEntities == null) {
       assertNull(retrievedEntityInfo.getRelatedEntities());
-    else
+    } else {
       assertEquals(relatedEntities, retrievedEntityInfo.getRelatedEntities());
-    if (primaryFilters == null)
+    }
+    if (primaryFilters == null) {
       assertNull(retrievedEntityInfo.getPrimaryFilters());
-    else
+    } else {
       assertTrue(primaryFilters.equals(
           retrievedEntityInfo.getPrimaryFilters()));
-    if (otherInfo == null)
+    }
+    if (otherInfo == null) {
       assertNull(retrievedEntityInfo.getOtherInfo());
-    else
+    } else {
       assertTrue(otherInfo.equals(retrievedEntityInfo.getOtherInfo()));
+    }
   }
 
   /**
    * Verify timeline events
    */
   private static void verifyEntityTimeline(
-      ATSEventsOfOneEntity retrievedEvents, String entity, String entityType,
-      ATSEvent... actualEvents) {
-    assertEquals(entity, retrievedEvents.getEntityId());
+      EventsOfOneEntity retrievedEvents, String entityId, String entityType,
+      TimelineEvent... actualEvents) {
+    assertEquals(entityId, retrievedEvents.getEntityId());
     assertEquals(entityType, retrievedEvents.getEntityType());
     assertEquals(actualEvents.length, retrievedEvents.getEvents().size());
     for (int i = 0; i < actualEvents.length; i++) {
@@ -503,32 +526,36 @@ public class ApplicationTimelineStoreTestUtils {
   /**
    * Create a test entity
    */
-  private static ATSEntity createEntity(String entity, String entityType,
-      Long startTime, List<ATSEvent> events,
-      Map<String, List<String>> relatedEntities,
-      Map<String, Object> primaryFilters, Map<String, Object> otherInfo) {
-    ATSEntity atsEntity = new ATSEntity();
-    atsEntity.setEntityId(entity);
-    atsEntity.setEntityType(entityType);
-    atsEntity.setStartTime(startTime);
-    atsEntity.setEvents(events);
-    if (relatedEntities != null)
-      for (Entry<String, List<String>> e : relatedEntities.entrySet())
-        for (String v : e.getValue())
-          atsEntity.addRelatedEntity(e.getKey(), v);
-    else
-      atsEntity.setRelatedEntities(null);
-    atsEntity.setPrimaryFilters(primaryFilters);
-    atsEntity.setOtherInfo(otherInfo);
-    return atsEntity;
+  protected static TimelineEntity createEntity(String entityId, String entityType,
+      Long startTime, List<TimelineEvent> events,
+      Map<String, Set<String>> relatedEntities,
+      Map<String, Set<Object>> primaryFilters,
+      Map<String, Object> otherInfo) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setEntityId(entityId);
+    entity.setEntityType(entityType);
+    entity.setStartTime(startTime);
+    entity.setEvents(events);
+    if (relatedEntities != null) {
+      for (Entry<String, Set<String>> e : relatedEntities.entrySet()) {
+        for (String v : e.getValue()) {
+          entity.addRelatedEntity(e.getKey(), v);
+        }
+      }
+    } else {
+      entity.setRelatedEntities(null);
+    }
+    entity.setPrimaryFilters(primaryFilters);
+    entity.setOtherInfo(otherInfo);
+    return entity;
   }
 
   /**
    * Create a test event
    */
-  private static ATSEvent createEvent(long timestamp, String type, Map<String,
+  private static TimelineEvent createEvent(long timestamp, String type, Map<String,
       Object> info) {
-    ATSEvent event = new ATSEvent();
+    TimelineEvent event = new TimelineEvent();
     event.setTimestamp(timestamp);
     event.setEventType(type);
     event.setEventInfo(info);

+ 44 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestATSWebServices.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestTimelineWebServices.java

@@ -24,13 +24,13 @@ import javax.ws.rs.core.MediaType;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
-import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.TestMemoryApplicationTimelineStore;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TestMemoryTimelineStore;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.junit.Test;
@@ -47,23 +47,23 @@ import com.sun.jersey.test.framework.JerseyTest;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 
 
-public class TestATSWebServices extends JerseyTest {
+public class TestTimelineWebServices extends JerseyTest {
 
-  private static ApplicationTimelineStore store;
+  private static TimelineStore store;
 
   private Injector injector = Guice.createInjector(new ServletModule() {
 
     @Override
     protected void configureServlets() {
       bind(YarnJacksonJaxbJsonProvider.class);
-      bind(ATSWebServices.class);
+      bind(TimelineWebServices.class);
       bind(GenericExceptionHandler.class);
       try{
-        store = mockApplicationTimelineStore();
+        store = mockTimelineStore();
       } catch (Exception e) {
         Assert.fail();
       }
-      bind(ApplicationTimelineStore.class).toInstance(store);
+      bind(TimelineStore.class).toInstance(store);
       serve("/*").with(GuiceContainer.class);
     }
 
@@ -77,15 +77,15 @@ public class TestATSWebServices extends JerseyTest {
     }
   }
 
-  private ApplicationTimelineStore mockApplicationTimelineStore()
+  private TimelineStore mockTimelineStore()
       throws Exception {
-    TestMemoryApplicationTimelineStore store =
-        new TestMemoryApplicationTimelineStore();
+    TestMemoryTimelineStore store =
+        new TestMemoryTimelineStore();
     store.setup();
-    return store.getApplicationTimelineStore();
+    return store.getTimelineStore();
   }
 
-  public TestATSWebServices() {
+  public TestTimelineWebServices() {
     super(new WebAppDescriptor.Builder(
         "org.apache.hadoop.yarn.server.applicationhistoryservice.webapp")
         .contextListenerClass(GuiceServletConfig.class)
@@ -99,28 +99,28 @@ public class TestATSWebServices extends JerseyTest {
   @Test
   public void testAbout() throws Exception {
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSWebServices.AboutInfo about =
-        response.getEntity(ATSWebServices.AboutInfo.class);
+    TimelineWebServices.AboutInfo about =
+        response.getEntity(TimelineWebServices.AboutInfo.class);
     Assert.assertNotNull(about);
-    Assert.assertEquals("Application Timeline API", about.getAbout());
+    Assert.assertEquals("Timeline API", about.getAbout());
   }
 
   @Test
   public void testGetEntities() throws Exception {
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .path("type_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSEntities entities = response.getEntity(ATSEntities.class);
+    TimelineEntities entities = response.getEntity(TimelineEntities.class);
     Assert.assertNotNull(entities);
     Assert.assertEquals(2, entities.getEntities().size());
-    ATSEntity entity1 = entities.getEntities().get(0);
+    TimelineEntity entity1 = entities.getEntities().get(0);
     Assert.assertNotNull(entity1);
     Assert.assertEquals("id_1", entity1.getEntityId());
     Assert.assertEquals("type_1", entity1.getEntityType());
@@ -128,7 +128,7 @@ public class TestATSWebServices extends JerseyTest {
     Assert.assertEquals(2, entity1.getEvents().size());
     Assert.assertEquals(2, entity1.getPrimaryFilters().size());
     Assert.assertEquals(4, entity1.getOtherInfo().size());
-    ATSEntity entity2 = entities.getEntities().get(1);
+    TimelineEntity entity2 = entities.getEntities().get(1);
     Assert.assertNotNull(entity2);
     Assert.assertEquals("id_2", entity2.getEntityId());
     Assert.assertEquals("type_1", entity2.getEntityType());
@@ -141,12 +141,12 @@ public class TestATSWebServices extends JerseyTest {
   @Test
   public void testGetEntity() throws Exception {
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .path("type_1").path("id_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSEntity entity = response.getEntity(ATSEntity.class);
+    TimelineEntity entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("id_1", entity.getEntityId());
     Assert.assertEquals("type_1", entity.getEntityType());
@@ -159,12 +159,12 @@ public class TestATSWebServices extends JerseyTest {
   @Test
   public void testGetEntityFields1() throws Exception {
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .path("type_1").path("id_1").queryParam("fields", "events,otherinfo")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSEntity entity = response.getEntity(ATSEntity.class);
+    TimelineEntity entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("id_1", entity.getEntityId());
     Assert.assertEquals("type_1", entity.getEntityType());
@@ -177,13 +177,13 @@ public class TestATSWebServices extends JerseyTest {
   @Test
   public void testGetEntityFields2() throws Exception {
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .path("type_1").path("id_1").queryParam("fields", "lasteventonly," +
             "primaryfilters,relatedentities")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSEntity entity = response.getEntity(ATSEntity.class);
+    TimelineEntity entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("id_1", entity.getEntityId());
     Assert.assertEquals("type_1", entity.getEntityType());
@@ -196,22 +196,22 @@ public class TestATSWebServices extends JerseyTest {
   @Test
   public void testGetEvents() throws Exception {
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .path("type_1").path("events")
         .queryParam("entityId", "id_1")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSEvents events = response.getEntity(ATSEvents.class);
+    TimelineEvents events = response.getEntity(TimelineEvents.class);
     Assert.assertNotNull(events);
     Assert.assertEquals(1, events.getAllEvents().size());
-    ATSEvents.ATSEventsOfOneEntity partEvents = events.getAllEvents().get(0);
+    TimelineEvents.EventsOfOneEntity partEvents = events.getAllEvents().get(0);
     Assert.assertEquals(2, partEvents.getEvents().size());
-    ATSEvent event1 = partEvents.getEvents().get(0);
+    TimelineEvent event1 = partEvents.getEvents().get(0);
     Assert.assertEquals(456l, event1.getTimestamp());
     Assert.assertEquals("end_event", event1.getEventType());
     Assert.assertEquals(1, event1.getEventInfo().size());
-    ATSEvent event2 = partEvents.getEvents().get(1);
+    TimelineEvent event2 = partEvents.getEvents().get(1);
     Assert.assertEquals(123l, event2.getTimestamp());
     Assert.assertEquals("start_event", event2.getEventType());
     Assert.assertEquals(0, event2.getEventInfo().size());
@@ -219,28 +219,28 @@ public class TestATSWebServices extends JerseyTest {
 
   @Test
   public void testPostEntities() throws Exception {
-    ATSEntities entities = new ATSEntities();
-    ATSEntity entity = new ATSEntity();
+    TimelineEntities entities = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
     entity.setEntityId("test id");
     entity.setEntityType("test type");
     entity.setStartTime(System.currentTimeMillis());
     entities.addEntity(entity);
     WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("apptimeline")
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ATSPutErrors errors = response.getEntity(ATSPutErrors.class);
-    Assert.assertNotNull(errors);
-    Assert.assertEquals(0, errors.getErrors().size());
+    TimelinePutResponse putResposne = response.getEntity(TimelinePutResponse.class);
+    Assert.assertNotNull(putResposne);
+    Assert.assertEquals(0, putResposne.getErrors().size());
     // verify the entity exists in the store
-    response = r.path("ws").path("v1").path("apptimeline")
+    response = r.path("ws").path("v1").path("timeline")
         .path("test type").path("test id")
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    entity = response.getEntity(ATSEntity.class);
+    entity = response.getEntity(TimelineEntity.class);
     Assert.assertNotNull(entity);
     Assert.assertEquals("test id", entity.getEntityId());
     Assert.assertEquals("test type", entity.getEntityType());

+ 10 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -224,11 +224,16 @@ public class NodeManager extends CompositeService
     new Thread() {
       @Override
       public void run() {
-        LOG.info("Notifying ContainerManager to block new container-requests");
-        containerManager.setBlockNewContainerRequests(true);
-        LOG.info("Cleaning up running containers on resync");
-        containerManager.cleanupContainersOnNMResync();
-        ((NodeStatusUpdaterImpl) nodeStatusUpdater ).rebootNodeStatusUpdater();
+        try {
+          LOG.info("Notifying ContainerManager to block new container-requests");
+          containerManager.setBlockNewContainerRequests(true);
+          LOG.info("Cleaning up running containers on resync");
+          containerManager.cleanupContainersOnNMResync();
+          ((NodeStatusUpdaterImpl) nodeStatusUpdater).rebootNodeStatusUpdater();
+        } catch (YarnRuntimeException e) {
+          LOG.fatal("Error while rebooting NodeStatusUpdater.", e);
+          shutDown();
+        }
       }
     }.start();
   }

+ 66 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
@@ -65,6 +66,7 @@ public class TestNodeManagerResync {
   private FileContext localFS;
   private CyclicBarrier syncBarrier;
   private AtomicBoolean assertionFailedInThread = new AtomicBoolean(false);
+  private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false);
 
   @Before
   public void setup() throws UnsupportedFileSystemException {
@@ -137,6 +139,30 @@ public class TestNodeManagerResync {
     Assert.assertFalse(assertionFailedInThread.get());
     nm.stop();
   }
+  
+  @SuppressWarnings("unchecked")
+  @Test(timeout=10000)
+  public void testNMshutdownWhenResyncThrowException() throws IOException,
+      InterruptedException, YarnException {
+    NodeManager nm = new TestNodeManager3();
+    YarnConfiguration conf = createNMConfig();
+    nm.init(conf);
+    nm.start();
+    Assert.assertEquals(1, ((TestNodeManager3) nm).getNMRegistrationCount());
+    nm.getNMDispatcher().getEventHandler()
+        .handle(new NodeManagerEvent(NodeManagerEventType.RESYNC));
+    
+    synchronized (isNMShutdownCalled) {
+      while (isNMShutdownCalled.get() == false) {
+        try {
+          isNMShutdownCalled.wait();
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+    
+    Assert.assertTrue("NM shutdown not called.",isNMShutdownCalled.get());
+  }
 
   private YarnConfiguration createNMConfig() {
     YarnConfiguration conf = new YarnConfiguration();
@@ -322,4 +348,44 @@ public class TestNodeManagerResync {
       }
     }
   }
+  
+  class TestNodeManager3 extends NodeManager {
+
+    private int registrationCount = 0;
+
+    @Override
+    protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+        Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
+      return new TestNodeStatusUpdaterImpl3(context, dispatcher, healthChecker,
+          metrics);
+    }
+
+    public int getNMRegistrationCount() {
+      return registrationCount;
+    }
+
+    @Override
+    protected void shutDown() {
+      synchronized (isNMShutdownCalled) {
+        isNMShutdownCalled.set(true);
+        isNMShutdownCalled.notify();
+      }
+    }
+
+    class TestNodeStatusUpdaterImpl3 extends MockNodeStatusUpdater {
+
+      public TestNodeStatusUpdaterImpl3(Context context, Dispatcher dispatcher,
+          NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
+        super(context, dispatcher, healthChecker, metrics);
+      }
+
+      @Override
+      protected void registerWithRM() throws YarnException, IOException {
+        super.registerWithRM();
+        registrationCount++;
+        if (registrationCount > 1) {
+          throw new YarnRuntimeException("Registration with RM failed.");
+        }
+      }
+    }}
 }

+ 23 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryWriter;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData;
@@ -81,8 +82,8 @@ public class RMApplicationHistoryWriter extends CompositeService {
   protected synchronized void serviceInit(Configuration conf) throws Exception {
 
     historyServiceEnabled =
-        conf.getBoolean(YarnConfiguration.YARN_HISTORY_SERVICE_ENABLED,
-          YarnConfiguration.DEFAULT_YARN_HISTORY_SERVICE_ENABLED);
+        conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
+          YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED);
 
     writer = createApplicationHistoryStore(conf);
     addIfService(writer);
@@ -112,14 +113,15 @@ public class RMApplicationHistoryWriter extends CompositeService {
     if (historyServiceEnabled) {
       try {
         Class<? extends ApplicationHistoryStore> storeClass =
-            conf.getClass(YarnConfiguration.RM_HISTORY_WRITER_CLASS,
-              NullApplicationHistoryStore.class, ApplicationHistoryStore.class);
+            conf.getClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
+              FileSystemApplicationHistoryStore.class,
+              ApplicationHistoryStore.class);
         return storeClass.newInstance();
       } catch (Exception e) {
         String msg =
             "Could not instantiate ApplicationHistoryWriter: "
-                + conf.get(YarnConfiguration.RM_HISTORY_WRITER_CLASS,
-                  NullApplicationHistoryStore.class.getName());
+                + conf.get(YarnConfiguration.APPLICATION_HISTORY_STORE,
+                  FileSystemApplicationHistoryStore.class.getName());
         LOG.error(msg, e);
         throw new YarnRuntimeException(msg, e);
       }
@@ -214,21 +216,25 @@ public class RMApplicationHistoryWriter extends CompositeService {
 
   @SuppressWarnings("unchecked")
   public void applicationStarted(RMApp app) {
-    dispatcher.getEventHandler().handle(
-      new WritingApplicationStartEvent(app.getApplicationId(),
-        ApplicationStartData.newInstance(app.getApplicationId(), app.getName(),
-          app.getApplicationType(), app.getQueue(), app.getUser(),
-          app.getSubmitTime(), app.getStartTime())));
+    if (historyServiceEnabled) {
+      dispatcher.getEventHandler().handle(
+        new WritingApplicationStartEvent(app.getApplicationId(),
+          ApplicationStartData.newInstance(app.getApplicationId(), app.getName(),
+            app.getApplicationType(), app.getQueue(), app.getUser(),
+            app.getSubmitTime(), app.getStartTime())));
+    }
   }
 
   @SuppressWarnings("unchecked")
   public void applicationFinished(RMApp app, RMAppState finalState) {
-    dispatcher.getEventHandler().handle(
-      new WritingApplicationFinishEvent(app.getApplicationId(),
-        ApplicationFinishData.newInstance(app.getApplicationId(),
-          app.getFinishTime(), app.getDiagnostics().toString(),
-          app.getFinalApplicationStatus(),
-          RMServerUtils.createApplicationState(finalState))));
+    if (historyServiceEnabled) {
+      dispatcher.getEventHandler().handle(
+        new WritingApplicationFinishEvent(app.getApplicationId(),
+          ApplicationFinishData.newInstance(app.getApplicationId(),
+            app.getFinishTime(), app.getDiagnostics().toString(),
+            app.getFinalApplicationStatus(),
+            RMServerUtils.createApplicationState(finalState))));
+    }
   }
 
   @SuppressWarnings("unchecked")

+ 42 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java

@@ -201,6 +201,22 @@ public class AppSchedulable extends Schedulable {
    * Assign a container to this node to facilitate {@code request}. If node does
    * not have enough memory, create a reservation. This is called once we are
    * sure the particular request should be facilitated by this node.
+   * 
+   * @param node
+   *     The node to try placing the container on.
+   * @param priority
+   *     The requested priority for the container.
+   * @param request
+   *     The ResourceRequest we're trying to satisfy.
+   * @param type
+   *     The locality of the assignment.
+   * @param reserved
+   *     Whether there's already a container reserved for this app on the node.
+   * @return
+   *     If an assignment was made, returns the resources allocated to the
+   *     container.  If a reservation was made, returns
+   *     FairScheduler.CONTAINER_RESERVED.  If no assignment or reservation was
+   *     made, returns an empty resource.
    */
   private Resource assignContainer(FSSchedulerNode node,
       ResourceRequest request, NodeType type,
@@ -255,17 +271,6 @@ public class AppSchedulable extends Schedulable {
       LOG.debug("Node offered to app: " + getName() + " reserved: " + reserved);
     }
 
-    if (reserved) {
-      RMContainer rmContainer = node.getReservedContainer();
-      Priority priority = rmContainer.getReservedPriority();
-
-      // Make sure the application still needs requests at this priority
-      if (app.getTotalRequiredResources(priority) == 0) {
-        unreserve(priority, node);
-        return Resources.none();
-      }
-    }
-
     Collection<Priority> prioritiesToTry = (reserved) ? 
         Arrays.asList(node.getReservedContainer().getReservedPriority()) : 
         app.getPriorities();
@@ -338,7 +343,33 @@ public class AppSchedulable extends Schedulable {
     return Resources.none();
   }
 
+  /**
+   * Called when this application already has an existing reservation on the
+   * given node.  Sees whether we can turn the reservation into an allocation.
+   * Also checks whether the application needs the reservation anymore, and
+   * releases it if not.
+   * 
+   * @param node
+   *     Node that the application has an existing reservation on
+   */
   public Resource assignReservedContainer(FSSchedulerNode node) {
+    RMContainer rmContainer = node.getReservedContainer();
+    Priority priority = rmContainer.getReservedPriority();
+
+    // Make sure the application still needs requests at this priority
+    if (app.getTotalRequiredResources(priority) == 0) {
+      unreserve(priority, node);
+      return Resources.none();
+    }
+    
+    // Fail early if the reserved container won't fit.
+    // Note that we have an assumption here that there's only one container size
+    // per priority.
+    if (!Resources.fitsIn(node.getReservedContainer().getReservedResource(),
+        node.getAvailableResource())) {
+      return Resources.none();
+    }
+    
     return assignContainer(node, true);
   }
 

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -1046,10 +1046,12 @@ public class FairScheduler extends AbstractYarnScheduler {
         reservedAppSchedulable = null;
       } else {
         // Reservation exists; try to fulfill the reservation
-        LOG.info("Trying to fulfill reservation for application "
-            + reservedAppSchedulable.getApp().getApplicationAttemptId()
-            + " on node: " + node);
-
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Trying to fulfill reservation for application "
+              + reservedAppSchedulable.getApp().getApplicationAttemptId()
+              + " on node: " + node);
+        }
+        
         node.getReservedAppSchedulable().assignReservedContainer(node);
       }
     }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java

@@ -77,7 +77,7 @@ public class TestRMApplicationHistoryWriter {
   public void setup() {
     store = new MemoryApplicationHistoryStore();
     Configuration conf = new Configuration();
-    conf.setBoolean(YarnConfiguration.YARN_HISTORY_SERVICE_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true);
     writer = new RMApplicationHistoryWriter() {
 
       @Override