Browse Source

Merging r1543111 through r1543509 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1543510 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 years ago
parent
commit
3c9cbc6893
60 changed files with 2413 additions and 366 deletions
  1. 5 0
      hadoop-common-project/hadoop-auth/pom.xml
  2. 9 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 69 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DirectCompressor.java
  4. 71 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DirectDecompressor.java
  5. 107 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java
  6. 105 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java
  7. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  8. 152 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java
  9. 13 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 2 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  11. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  12. 78 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  13. 23 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  14. 18 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  15. 10 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  16. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  17. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/TableListing.java
  18. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  19. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  20. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/css/bootstrap.min.css
  21. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.eot
  22. 229 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.svg
  23. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.ttf
  24. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.woff
  25. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/js/bootstrap.min.js
  26. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js
  27. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
  28. 8 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
  29. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java
  30. 97 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
  31. 17 0
      hadoop-yarn-project/CHANGES.txt
  32. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  33. 21 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
  34. 29 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
  35. 16 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
  36. 98 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerRequestPBImpl.java
  37. 17 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
  38. 16 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
  39. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  40. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java
  41. 46 50
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  42. 24 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  43. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  44. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  45. 96 52
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  46. 35 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  47. 33 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  48. 32 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  49. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptState.java
  50. 190 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
  51. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
  52. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
  53. 177 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/SchedulerPageUtil.java
  54. 10 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  55. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  56. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  57. 253 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  58. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  59. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
  60. 212 84
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java

+ 5 - 0
hadoop-common-project/hadoop-auth/pom.xml

@@ -52,6 +52,11 @@
       <artifactId>mockito-all</artifactId>
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
       <scope>test</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
     <dependency>
       <groupId>org.mortbay.jetty</groupId>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty</artifactId>
       <artifactId>jetty</artifactId>

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

@@ -387,6 +387,9 @@ Release 2.3.0 - UNRELEASED
 
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
 
 
+    HADOOP-10047. Add a direct-buffer based apis for compression. (Gopal V via
+    acmurthy)
+
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-9964. Fix deadlocks in TestHttpServer by synchronize
     HADOOP-9964. Fix deadlocks in TestHttpServer by synchronize
@@ -439,6 +442,9 @@ Release 2.3.0 - UNRELEASED
 
 
     HADOOP-10100. MiniKDC shouldn't use apacheds-all artifact. (rkanter via tucu)
     HADOOP-10100. MiniKDC shouldn't use apacheds-all artifact. (rkanter via tucu)
 
 
+    HADOOP-10107. Server.getNumOpenConnections may throw NPE. (Kihwal Lee via
+    jing9)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -491,6 +497,9 @@ Release 2.2.1 - UNRELEASED
 
 
     HADOOP-10078. KerberosAuthenticator always does SPNEGO. (rkanter via tucu)
     HADOOP-10078. KerberosAuthenticator always does SPNEGO. (rkanter via tucu)
 
 
+    HADOOP-10110. hadoop-auth has a build break due to missing dependency.
+    (Chuan Liu via arp)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 69 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DirectCompressor.java

@@ -0,0 +1,69 @@
+/**
+ * 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.io.compress;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface DirectCompressor extends Compressor {
+  /**
+   * Example usage
+   * <pre> {@code
+   * private void compress(DirectCompressor comp, ByteBufferProducer in, ByteBufferConsumer out) throws IOException {
+   *    ByteBuffer outBB = ByteBuffer.allocateDirect(64*1024);
+   *    outBB.clear();
+   *    // returns inBB.remaining() &gt; 0 || inBB == null 
+   *    // if you do a inBB.put(), remember to do a inBB.flip()
+   *    ByteBuffer inBB = in.get();       
+   *    while(!comp.finished()) {
+   *      comp.compress(outBB, inBB);
+   *      if(outBB.remaining() == 0) {
+   *        // flush when the buffer only when it is full
+   *        outBB.flip();          
+   *        // has to consume the buffer, because it is reused
+   *        out.put(outBB);
+   *        outBB.clear();
+   *      }
+   *      if(inBB != null &amp;&amp; inBB.remaining() == 0) {
+   *        inBB = in.get();
+   *        if(inBB == null) {
+   *          // EOF
+   *          comp.finish();
+   *        }
+   *      }
+   *    }
+   *    
+   *    if(outBB.position() &gt; 0) {
+   *      outBB.flip();
+   *      out.put(outBB);
+   *      outBB.clear();
+   *    }
+   *  }
+   * } </pre>
+   * @param dst Destination {@link ByteBuffer} for storing the results into. Requires dst.remaining() to be > 0
+   * @param src Source {@link ByteBuffer} for reading from. This can be null or src.remaining() > 0
+   * @return bytes stored into dst
+   * @throws IOException if compression fails
+   */
+	public int compress(ByteBuffer dst, ByteBuffer src) throws IOException;
+}

+ 71 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DirectDecompressor.java

@@ -0,0 +1,71 @@
+/**
+ * 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.io.compress;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface DirectDecompressor extends Decompressor {
+  /**
+   * Example usage
+   * 
+   * <pre>{@code
+   * private void decompress(DirectDecompressor decomp, ByteBufferProducer in, ByteBufferConsumer out) throws IOException {
+   *    ByteBuffer outBB = ByteBuffer.allocate(64*1024);
+   *    outBB.clear();
+   *    // returns inBB.remaining() &gt; 0 || inBB == null 
+   *    // if you do a inBB.put(), remember to do a inBB.flip()
+   *    ByteBuffer inBB = in.get();
+   *    if(inBB == null) {
+   *      // no data at all?
+   *    }
+   *    while(!decomp.finished()) {
+   *      decomp.decompress(outBB, inBB);
+   *      if(outBB.remaining() == 0) {
+   *        // flush when the buffer is full
+   *        outBB.flip();
+   *        // has to consume the buffer, because it is reused
+   *        out.put(outBB);
+   *        outBB.clear();
+   *      }
+   *      if(inBB != null &amp;&amp; inBB.remaining() == 0) {
+   *        // inBB = null for EOF
+   *        inBB = in.get();
+   *      }
+   *    }
+   *    
+   *    if(outBB.position() &gt; 0) {
+   *      outBB.flip();
+   *      out.put(outBB);
+   *      outBB.clear();
+   *    }
+   *  }
+   * }</pre>
+   * @param dst Destination {@link ByteBuffer} for storing the results into. Requires dst.remaining() to be > 0
+   * @param src Source {@link ByteBuffer} for reading from. This can be null or src.remaining() > 0
+   * @return bytes stored into dst (dst.postion += more)
+   * @throws IOException if compression fails   
+   */
+	public int decompress(ByteBuffer dst, ByteBuffer src) throws IOException;
+}

+ 107 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java

@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.Compressor;
 import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.DirectCompressor;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.NativeCodeLoader;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -35,7 +36,7 @@ import org.apache.commons.logging.LogFactory;
  * http://www.zlib.net/
  * http://www.zlib.net/
  * 
  * 
  */
  */
-public class ZlibCompressor implements Compressor {
+public class ZlibCompressor implements Compressor,DirectCompressor {
 
 
   private static final Log LOG = LogFactory.getLog(ZlibCompressor.class);
   private static final Log LOG = LogFactory.getLog(ZlibCompressor.class);
 
 
@@ -420,6 +421,7 @@ public class ZlibCompressor implements Compressor {
     compressedDirectBuf.limit(directBufferSize);
     compressedDirectBuf.limit(directBufferSize);
     compressedDirectBuf.position(directBufferSize);
     compressedDirectBuf.position(directBufferSize);
     userBufOff = userBufLen = 0;
     userBufOff = userBufLen = 0;
+    userBuf = null;
   }
   }
   
   
   @Override
   @Override
@@ -435,6 +437,110 @@ public class ZlibCompressor implements Compressor {
       throw new NullPointerException();
       throw new NullPointerException();
   }
   }
   
   
+  private int put(ByteBuffer dst, ByteBuffer src) {
+    // this will lop off data from src[pos:limit] into dst[pos:limit]
+    int l1 = src.remaining();
+    int l2 = dst.remaining();
+    int pos1 = src.position();
+    int pos2 = dst.position();
+    int len = Math.min(l1, l2);
+
+    if (len == 0) {
+      return 0;
+    }
+
+    ByteBuffer slice = src.slice();
+    slice.limit(len);
+    dst.put(slice);
+    src.position(pos1 + len);
+    return len;
+  }
+
+  public int compress(ByteBuffer dst, ByteBuffer src) throws IOException {
+    assert dst.remaining() > 0 : "dst.remaining() == 0";
+    int n = 0;
+    
+    /* fast path for clean state and direct buffers */
+    /* TODO: reset should free userBuf? */
+    if((src != null && src.isDirect()) && dst.isDirect() && userBuf == null) {
+      /*
+       * TODO: fix these assumptions in inflateDirect(), eventually by allowing
+       * it to read position()/limit() directly
+       */
+      boolean cleanDst = (dst.position() == 0 && dst.remaining() == dst.capacity() && dst.capacity() >= directBufferSize);
+      boolean cleanState = (keepUncompressedBuf == false && uncompressedDirectBufLen == 0 && compressedDirectBuf.remaining() == 0);
+      /* use the buffers directly */
+      if(cleanDst && cleanState) {
+        Buffer originalCompressed = compressedDirectBuf;
+        Buffer originalUncompressed = uncompressedDirectBuf;
+        int originalBufferSize = directBufferSize;
+        uncompressedDirectBuf = src;
+        uncompressedDirectBufOff = src.position();
+        uncompressedDirectBufLen = src.remaining();
+        compressedDirectBuf = dst;
+        directBufferSize = dst.remaining();
+        // Compress data
+        n = deflateBytesDirect();
+        // we move dst.position() forward, not limit() 
+        // unlike the local buffer case, which moves it when we put() into the dst
+        dst.position(n);
+        if(uncompressedDirectBufLen > 0) {
+          src.position(uncompressedDirectBufOff);
+        } else {
+          src.position(src.limit());
+        }
+        compressedDirectBuf = originalCompressed;
+        uncompressedDirectBuf = originalUncompressed;
+        uncompressedDirectBufOff = 0;
+        uncompressedDirectBufLen = 0;
+        directBufferSize = originalBufferSize;
+        return n;
+      }
+    }
+    
+    // Check if there is compressed data
+    if (compressedDirectBuf.remaining() > 0) {
+      n = put(dst, (ByteBuffer) compressedDirectBuf);
+    }
+
+    if (dst.remaining() == 0) {
+      return n;
+    } else {
+      needsInput();
+
+      // if we have drained userBuf, read from src (ideally, do not mix buffer
+      // modes, but sometimes you can)
+      if (userBufLen == 0 && src != null && src.remaining() > 0) {
+        put((ByteBuffer) uncompressedDirectBuf, src);
+        uncompressedDirectBufLen = uncompressedDirectBuf.position();
+      }
+
+      // Re-initialize the zlib's output direct buffer
+      compressedDirectBuf.rewind();
+      compressedDirectBuf.limit(directBufferSize);
+
+      // Compress data
+      int more = deflateBytesDirect();
+
+      compressedDirectBuf.limit(more);
+
+      // Check if zlib consumed all input buffer
+      // set keepUncompressedBuf properly
+      if (uncompressedDirectBufLen <= 0) { // zlib consumed all input buffer
+        keepUncompressedBuf = false;
+        uncompressedDirectBuf.clear();
+        uncompressedDirectBufOff = 0;
+        uncompressedDirectBufLen = 0;
+      } else { // zlib did not consume all input buffer
+        keepUncompressedBuf = true;
+      }
+
+      // fill the dst buffer from compressedDirectBuf
+      int fill = put(dst, ((ByteBuffer) compressedDirectBuf));
+      return n + fill;
+    }
+  }
+  
   private native static void initIDs();
   private native static void initIDs();
   private native static long init(int level, int strategy, int windowBits);
   private native static long init(int level, int strategy, int windowBits);
   private native static void setDictionary(long strm, byte[] b, int off,
   private native static void setDictionary(long strm, byte[] b, int off,

+ 105 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java

@@ -23,6 +23,7 @@ import java.nio.Buffer;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 
 
 import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DirectDecompressor;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.NativeCodeLoader;
 
 
 /**
 /**
@@ -31,7 +32,7 @@ import org.apache.hadoop.util.NativeCodeLoader;
  * http://www.zlib.net/
  * http://www.zlib.net/
  * 
  * 
  */
  */
-public class ZlibDecompressor implements Decompressor {
+public class ZlibDecompressor implements Decompressor,DirectDecompressor {
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64*1024;
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64*1024;
   
   
   // HACK - Use this as a global lock in the JNI layer
   // HACK - Use this as a global lock in the JNI layer
@@ -280,6 +281,7 @@ public class ZlibDecompressor implements Decompressor {
     uncompressedDirectBuf.limit(directBufferSize);
     uncompressedDirectBuf.limit(directBufferSize);
     uncompressedDirectBuf.position(directBufferSize);
     uncompressedDirectBuf.position(directBufferSize);
     userBufOff = userBufLen = 0;
     userBufOff = userBufLen = 0;
+    userBuf = null;
   }
   }
 
 
   @Override
   @Override
@@ -299,6 +301,108 @@ public class ZlibDecompressor implements Decompressor {
     if (stream == 0)
     if (stream == 0)
       throw new NullPointerException();
       throw new NullPointerException();
   }
   }
+    
+  private int put(ByteBuffer dst, ByteBuffer src) {
+    // this will lop off data from src[pos:limit] into dst[pos:limit], using the
+    // min() of both remaining()
+    int l1 = src.remaining();
+    int l2 = dst.remaining();
+    int pos1 = src.position();
+    int pos2 = dst.position();
+    int len = Math.min(l1, l2);
+
+    if (len == 0) {
+      return 0;
+    }
+
+    ByteBuffer slice = src.slice();
+    slice.limit(len);
+    dst.put(slice);
+    src.position(pos1 + len);
+    return len;
+  }
+
+  public int decompress(ByteBuffer dst, ByteBuffer src) throws IOException {
+    assert dst.remaining() > 0 : "dst.remaining == 0";
+    int n = 0;
+    
+    /* fast path for clean state and direct buffers */
+    if((src != null && src.isDirect()) && dst.isDirect() && userBuf == null) {
+      /*
+       * TODO: fix these assumptions in inflateDirect(), eventually by allowing
+       * it to read position()/limit() directly
+       */
+      boolean cleanDst = (dst.position() == 0 && dst.remaining() == dst.capacity() && dst.remaining() >= directBufferSize);
+      boolean cleanState = (compressedDirectBufLen == 0 && uncompressedDirectBuf.remaining() == 0);
+      /* use the buffers directly */
+      if(cleanDst && cleanState) {
+        Buffer originalCompressed = compressedDirectBuf;
+        Buffer originalUncompressed = uncompressedDirectBuf;
+        int originalBufferSize = directBufferSize;
+        compressedDirectBuf = src;
+        compressedDirectBufOff = src.position();
+        compressedDirectBufLen = src.remaining();
+        uncompressedDirectBuf = dst;
+        directBufferSize = dst.remaining();
+        // Compress data
+        n = inflateBytesDirect();
+        dst.position(n);
+        if(compressedDirectBufLen > 0) {
+          src.position(compressedDirectBufOff);
+        } else {
+          src.position(src.limit());
+        }
+        compressedDirectBuf = originalCompressed;
+        uncompressedDirectBuf = originalUncompressed;        
+        compressedDirectBufOff = 0;
+        compressedDirectBufLen = 0;
+        directBufferSize = originalBufferSize;
+        return n;
+      }
+    }
+    
+    // Check if there is compressed data
+    if (uncompressedDirectBuf.remaining() > 0) {
+      n = put(dst, (ByteBuffer) uncompressedDirectBuf);
+    }
+
+    if (dst.remaining() == 0) {
+      return n;
+    } else {
+      if (needsInput()) {
+        // this does not update buffers if we have no userBuf
+        if (userBufLen <= 0) {
+          compressedDirectBufOff = 0;
+          compressedDirectBufLen = 0;
+          compressedDirectBuf.rewind().limit(directBufferSize);
+        }
+        if (src != null) {
+          assert src.remaining() > 0 : "src.remaining() == 0";
+        }
+      }
+
+      // if we have drained userBuf, read from src (ideally, do not mix buffer
+      // modes, but sometimes you can)
+      if (userBufLen == 0 && src != null && src.remaining() > 0) {
+        compressedDirectBufLen += put(((ByteBuffer) compressedDirectBuf), src);
+      }
+      
+      // Re-initialize the zlib's output direct buffer
+      uncompressedDirectBuf.rewind();
+      uncompressedDirectBuf.limit(directBufferSize);
+
+      // Compress data
+      int more = inflateBytesDirect();
+
+      uncompressedDirectBuf.limit(more);
+
+      // Get atmost 'len' bytes
+      int fill = put(dst, ((ByteBuffer) uncompressedDirectBuf));
+      return n + fill;
+    }
+  }
+
+  
   
   
   private native static void initIDs();
   private native static void initIDs();
   private native static long init(int windowBits);
   private native static long init(int windowBits);

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -2109,6 +2109,7 @@ public abstract class Server {
     // Start the listener here and let it bind to the port
     // Start the listener here and let it bind to the port
     listener = new Listener();
     listener = new Listener();
     this.port = listener.getAddress().getPort();    
     this.port = listener.getAddress().getPort();    
+    connectionManager = new ConnectionManager();
     this.rpcMetrics = RpcMetrics.create(this);
     this.rpcMetrics = RpcMetrics.create(this);
     this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port);
     this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port);
     this.tcpNoDelay = conf.getBoolean(
     this.tcpNoDelay = conf.getBoolean(
@@ -2117,7 +2118,6 @@ public abstract class Server {
 
 
     // Create the responder here
     // Create the responder here
     responder = new Responder();
     responder = new Responder();
-    connectionManager = new ConnectionManager();
     
     
     if (secretManager != null) {
     if (secretManager != null) {
       SaslRpcServer.init(conf);
       SaslRpcServer.init(conf);

+ 152 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java

@@ -19,8 +19,13 @@ package org.apache.hadoop.io.compress.zlib;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 import static org.junit.Assume.*;
 import static org.junit.Assume.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Console;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
+import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -33,8 +38,12 @@ import org.apache.hadoop.io.compress.DecompressorStream;
 import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStrategy;
 import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStrategy;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
+import org.apache.log4j.ConsoleAppender;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+
+import sun.util.logging.resources.logging;
+
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSet;
 
 
 public class TestZlibCompressorDecompressor {
 public class TestZlibCompressorDecompressor {
@@ -150,6 +159,149 @@ public class TestZlibCompressorDecompressor {
     }
     }
   }
   }
   
   
+  private void compressDecompressLoop(int rawDataSize, int inSize, int outSize)
+      throws IOException {
+    byte[] rawData = null;
+    rawData = generate(rawDataSize);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    ByteBuffer inBuf = ByteBuffer.allocateDirect(inSize);
+    ByteBuffer outBuf = ByteBuffer.allocateDirect(outSize);
+    ZlibCompressor compressor = new ZlibCompressor();
+    ZlibDecompressor decompressor = new ZlibDecompressor();
+    outBuf.clear();
+    /* compression loop */
+    int off = 0;
+    int len = rawDataSize;
+    int min = Math.min(inBuf.remaining(), len);
+    if (min > 0) {
+      inBuf.put(rawData, off, min);
+    }
+    inBuf.flip();
+    len -= min;
+    off += min;
+    while (!compressor.finished()) {
+      compressor.compress(outBuf, inBuf);
+      if (outBuf.remaining() == 0) {
+        // flush when the buffer is full
+        outBuf.flip();
+        while (outBuf.remaining() > 0) {
+          baos.write(outBuf.get());
+        }
+        outBuf.clear();
+      }
+      if (inBuf != null && inBuf.remaining() == 0) {
+        inBuf.clear();
+        if (len > 0) {
+          min = Math.min(inBuf.remaining(), len);
+          inBuf.put(rawData, off, min);
+          inBuf.flip();
+          len -= min;
+          off += min;
+        } else {
+          inBuf = null;
+          compressor.finish();
+        }
+      }
+    }
+
+    outBuf.flip();
+    if (outBuf.remaining() > 0) {
+      while (outBuf.remaining() > 0) {
+        baos.write(outBuf.get());
+      }
+      outBuf.clear();
+    }
+
+    compressor.end();
+
+    byte[] compressed = baos.toByteArray();
+    ByteBuffer expected = ByteBuffer.wrap(rawData);
+    outBuf.clear();
+    inBuf = ByteBuffer.allocateDirect(inSize);
+    inBuf.clear();
+
+    // zlib always has header
+    if (compressed.length != 0) {
+      off = 0;
+      len = compressed.length;
+      min = Math.min(inBuf.remaining(), len);
+      inBuf.put(compressed, off, min);
+      inBuf.flip();
+      len -= min;
+      off += min;
+      while (!decompressor.finished()) {
+        decompressor.decompress(outBuf, inBuf);
+        if (outBuf.remaining() == 0) {
+          outBuf.flip();
+          while (outBuf.remaining() > 0) {
+            assertEquals(expected.get(), outBuf.get());
+          }
+          outBuf.clear();
+        }
+
+        if (inBuf != null && inBuf.remaining() == 0) {
+          inBuf.clear();
+          if (len > 0) {
+            min = Math.min(inBuf.remaining(), len);
+            inBuf.put(compressed, off, min);
+            inBuf.flip();
+            len -= min;
+            off += min;
+          }
+        }
+      }
+    }
+
+    outBuf.flip();
+    if (outBuf.remaining() > 0) {
+      while (outBuf.remaining() > 0) {
+        assertEquals(expected.get(), outBuf.get());
+      }
+      outBuf.clear();
+    }
+
+    assertEquals(0, expected.remaining());
+  }
+  
+  @Test
+  public void testZlibDirectCompressDecompress() {
+    int[] size = { 4, 16, 4 * 1024, 64 * 1024, 128 * 1024, 256 * 1024,
+        1024 * 1024 };
+    try {
+      // 0-2 bytes results in sizeof(outBuf) > sizeof(inBuf)
+      compressDecompressLoop(0, 4096, 4096);
+      compressDecompressLoop(0, 1, 1);
+      compressDecompressLoop(1, 1, 2);
+      compressDecompressLoop(1, 2, 1);
+      compressDecompressLoop(2, 3, 2);
+
+      for (int i = 0; i < size.length; i++) {
+        compressDecompressLoop(size[i], 4096, 4096);
+        compressDecompressLoop(size[i], 1, 1);
+        compressDecompressLoop(size[i], 1, 2);
+        compressDecompressLoop(size[i], 2, 1);
+        compressDecompressLoop(size[i], 3, 2);
+        compressDecompressLoop(size[i], size[i], 4096);
+        compressDecompressLoop(size[i], size[i] - 1, 4096);
+        compressDecompressLoop(size[i], size[i] + 1, 4096);
+        compressDecompressLoop(size[i], 4096, size[i]);
+        compressDecompressLoop(size[i], 4096, size[i] - 1);
+        compressDecompressLoop(size[i], 4096, size[i] + 1);
+        compressDecompressLoop(size[i], size[i] - 1, size[i] - 1);
+
+        compressDecompressLoop(size[i], size[i] / 2, 4096);
+        compressDecompressLoop(size[i], size[i] / 2 - 1, 4096);
+        compressDecompressLoop(size[i], size[i] / 2 + 1, 4096);
+        compressDecompressLoop(size[i], 4096, size[i] / 2);
+        compressDecompressLoop(size[i], 4096, size[i] / 2 - 1);
+        compressDecompressLoop(size[i], 4096, size[i] / 2 + 1);
+        compressDecompressLoop(size[i], size[i] / 2 - 1, size[i] / 2 - 1);
+      }
+    } catch (IOException ex) {
+      fail("testZlibDirectCompressDecompress ex !!!" + ex);
+    }
+  }
+  
   @Test
   @Test
   public void testZlibCompressorDecompressorSetDictionary() {
   public void testZlibCompressorDecompressorSetDictionary() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();

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

@@ -375,6 +375,12 @@ Trunk (Unreleased)
 
 
     HDFS-5320. Add datanode caching metrics. (wang)
     HDFS-5320. Add datanode caching metrics. (wang)
 
 
+    HDFS-5520. loading cache path directives from edit log doesn't update
+    nextEntryId (cmccabe)
+
+    HDFS-5512. CacheAdmin -listPools fails with NPE when user lacks permissions
+    to view all pools (awang via cmccabe)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -496,6 +502,10 @@ Release 2.3.0 - UNRELEASED
 
 
     HDFS-5489. Use TokenAspect in WebHDFSFileSystem. (Haohui Mai via jing9)
     HDFS-5489. Use TokenAspect in WebHDFSFileSystem. (Haohui Mai via jing9)
 
 
+    HDFS-5393. Serve bootstrap and jQuery locally. (Haohui Mai via jing9)
+
+    HDFS-5073. TestListCorruptFileBlocks fails intermittently. (Arpit Agarwal)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -566,6 +576,9 @@ Release 2.3.0 - UNRELEASED
 
 
     HDFS-5502. Fix HTTPS support in HsftpFileSystem. (Haohui Mai via jing9)
     HDFS-5502. Fix HTTPS support in HsftpFileSystem. (Haohui Mai via jing9)
 
 
+    HDFS-5428. Under construction files deletion after snapshot+checkpoint+nn restart 
+    leads nn safemode. (jing9)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -547,8 +547,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/docs/releasenotes.html</exclude>
             <exclude>src/main/docs/releasenotes.html</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
             <exclude>src/site/resources/images/*</exclude>
+            <exclude>src/main/webapps/static/bootstrap-3.0.2/**</exclude>
             <exclude>src/main/webapps/static/dust-full-2.0.0.min.js</exclude>
             <exclude>src/main/webapps/static/dust-full-2.0.0.min.js</exclude>
             <exclude>src/main/webapps/static/dust-helpers-1.1.1.min.js</exclude>
             <exclude>src/main/webapps/static/dust-helpers-1.1.1.min.js</exclude>
+            <exclude>src/main/webapps/static/jquery-1.10.2.min.js</exclude>
             <exclude>src/main/webapps/hdfs/dfshealth.dust.html</exclude>
             <exclude>src/main/webapps/hdfs/dfshealth.dust.html</exclude>
             <exclude>src/main/webapps/hdfs/explorer-block-info.dust.html</exclude>
             <exclude>src/main/webapps/hdfs/explorer-block-info.dust.html</exclude>
             <exclude>src/main/webapps/hdfs/explorer.dust.html</exclude>
             <exclude>src/main/webapps/hdfs/explorer.dust.html</exclude>

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -249,7 +249,7 @@ public final class CacheManager {
 
 
   private long getNextEntryId() throws IOException {
   private long getNextEntryId() throws IOException {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
-    if (nextEntryId == Long.MAX_VALUE) {
+    if (nextEntryId >= Long.MAX_VALUE - 1) {
       throw new IOException("No more available IDs.");
       throw new IOException("No more available IDs.");
     }
     }
     return nextEntryId++;
     return nextEntryId++;
@@ -357,6 +357,17 @@ public final class CacheManager {
         // We are loading an entry from the edit log.
         // We are loading an entry from the edit log.
         // Use the ID from the edit log.
         // Use the ID from the edit log.
         id = directive.getId();
         id = directive.getId();
+        if (id <= 0) {
+          throw new InvalidRequestException("can't add an ID " +
+              "of " + id + ": it is not positive.");
+        }
+        if (id >= Long.MAX_VALUE) {
+          throw new InvalidRequestException("can't add an ID " +
+              "of " + id + ": it is too big.");
+        }
+        if (nextEntryId <= id) {
+          nextEntryId = id + 1;
+        }
       } else {
       } else {
         // Add a new entry with the next available ID.
         // Add a new entry with the next available ID.
         id = getNextEntryId();
         id = getNextEntryId();

+ 78 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -32,6 +32,7 @@ import java.security.DigestOutputStream;
 import java.security.MessageDigest;
 import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
@@ -678,6 +680,12 @@ public class FSImageFormat {
           if (underConstruction) {
           if (underConstruction) {
             clientName = FSImageSerialization.readString(in);
             clientName = FSImageSerialization.readString(in);
             clientMachine = FSImageSerialization.readString(in);
             clientMachine = FSImageSerialization.readString(in);
+            // convert the last block to BlockUC
+            if (blocks != null && blocks.length > 0) {
+              BlockInfo lastBlk = blocks[blocks.length - 1]; 
+              blocks[blocks.length - 1] = new BlockInfoUnderConstruction(
+                  lastBlk, replication);
+            }
           }
           }
         }
         }
       }
       }
@@ -690,10 +698,15 @@ public class FSImageFormat {
       }
       }
       final INodeFile file = new INodeFile(inodeId, localName, permissions,
       final INodeFile file = new INodeFile(inodeId, localName, permissions,
           modificationTime, atime, blocks, replication, blockSize);
           modificationTime, atime, blocks, replication, blockSize);
-      return fileDiffs != null? new INodeFileWithSnapshot(file, fileDiffs)
-          : underConstruction? new INodeFileUnderConstruction(
-              file, clientName, clientMachine, null)
-          : file;
+      if (underConstruction) {
+        INodeFileUnderConstruction fileUC = new INodeFileUnderConstruction(
+            file, clientName, clientMachine, null);
+        return fileDiffs == null ? fileUC :
+          new INodeFileUnderConstructionWithSnapshot(fileUC, fileDiffs);
+      } else {
+        return fileDiffs == null ? file : 
+          new INodeFileWithSnapshot(file, fileDiffs);
+      }
     } else if (numBlocks == -1) {
     } else if (numBlocks == -1) {
       //directory
       //directory
       
       
@@ -821,8 +834,20 @@ public class FSImageFormat {
 
 
         // verify that file exists in namespace
         // verify that file exists in namespace
         String path = cons.getLocalName();
         String path = cons.getLocalName();
-        final INodesInPath iip = fsDir.getLastINodeInPath(path);
-        INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
+        INodeFile oldnode = null;
+        boolean inSnapshot = false;
+        if (path != null && FSDirectory.isReservedName(path) && 
+            LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
+          // TODO: for HDFS-5428, we use reserved path for those INodeFileUC in 
+          // snapshot. If we support INode ID in the layout version, we can use
+          // the inode id to find the oldnode.
+          oldnode = namesystem.dir.getInode(cons.getId()).asFile();
+          inSnapshot = true;
+        } else {
+          final INodesInPath iip = fsDir.getLastINodeInPath(path);
+          oldnode = INodeFile.valueOf(iip.getINode(0), path);
+        }
+        
         cons.setLocalName(oldnode.getLocalNameBytes());
         cons.setLocalName(oldnode.getLocalNameBytes());
         INodeReference parentRef = oldnode.getParentReference();
         INodeReference parentRef = oldnode.getParentReference();
         if (parentRef != null) {
         if (parentRef != null) {
@@ -833,11 +858,23 @@ public class FSImageFormat {
 
 
         if (oldnode instanceof INodeFileWithSnapshot) {
         if (oldnode instanceof INodeFileWithSnapshot) {
           cons = new INodeFileUnderConstructionWithSnapshot(cons,
           cons = new INodeFileUnderConstructionWithSnapshot(cons,
-              ((INodeFileWithSnapshot)oldnode).getDiffs());
+              ((INodeFileWithSnapshot) oldnode).getDiffs());
         }
         }
 
 
-        fsDir.replaceINodeFile(path, oldnode, cons);
-        namesystem.leaseManager.addLease(cons.getClientName(), path); 
+        if (!inSnapshot) {
+          fsDir.replaceINodeFile(path, oldnode, cons);
+          namesystem.leaseManager.addLease(cons.getClientName(), path);
+        } else {
+          if (parentRef != null) {
+            // replace oldnode with cons
+            parentRef.setReferredINode(cons);
+          } else {
+            // replace old node in its parent's children list and deleted list
+            oldnode.getParent().replaceChildFileInSnapshot(oldnode, cons);
+            namesystem.dir.addToInodeMap(cons);
+            updateBlocksMap(cons);
+          }
+        }
       }
       }
     }
     }
 
 
@@ -916,6 +953,9 @@ public class FSImageFormat {
     /** The MD5 checksum of the file that was written */
     /** The MD5 checksum of the file that was written */
     private MD5Hash savedDigest;
     private MD5Hash savedDigest;
     private final ReferenceMap referenceMap = new ReferenceMap();
     private final ReferenceMap referenceMap = new ReferenceMap();
+    
+    private final Map<Long, INodeFileUnderConstruction> snapshotUCMap = 
+        new HashMap<Long, INodeFileUnderConstruction>();
 
 
     /** @throws IllegalStateException if the instance has not yet saved an image */
     /** @throws IllegalStateException if the instance has not yet saved an image */
     private void checkSaved() {
     private void checkSaved() {
@@ -992,14 +1032,22 @@ public class FSImageFormat {
         // save the root
         // save the root
         saveINode2Image(fsDir.rootDir, out, false, referenceMap, counter);
         saveINode2Image(fsDir.rootDir, out, false, referenceMap, counter);
         // save the rest of the nodes
         // save the rest of the nodes
-        saveImage(fsDir.rootDir, out, true, counter);
+        saveImage(fsDir.rootDir, out, true, false, counter);
         prog.endStep(Phase.SAVING_CHECKPOINT, step);
         prog.endStep(Phase.SAVING_CHECKPOINT, step);
         // Now that the step is finished, set counter equal to total to adjust
         // Now that the step is finished, set counter equal to total to adjust
         // for possible under-counting due to reference inodes.
         // for possible under-counting due to reference inodes.
         prog.setCount(Phase.SAVING_CHECKPOINT, step,
         prog.setCount(Phase.SAVING_CHECKPOINT, step,
           fsDir.rootDir.numItemsInTree());
           fsDir.rootDir.numItemsInTree());
         // save files under construction
         // save files under construction
-        sourceNamesystem.saveFilesUnderConstruction(out);
+        // TODO: for HDFS-5428, since we cannot break the compatibility of 
+        // fsimage, we store part of the under-construction files that are only
+        // in snapshots in this "under-construction-file" section. As a 
+        // temporary solution, we use "/.reserved/.inodes/<inodeid>" as their 
+        // paths, so that when loading fsimage we do not put them into the lease
+        // map. In the future, we can remove this hack when we can bump the 
+        // layout version.
+        sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap);
+        
         context.checkCancelled();
         context.checkCancelled();
         sourceNamesystem.saveSecretManagerState(out, sdPath);
         sourceNamesystem.saveSecretManagerState(out, sdPath);
         context.checkCancelled();
         context.checkCancelled();
@@ -1024,20 +1072,31 @@ public class FSImageFormat {
      * Save children INodes.
      * Save children INodes.
      * @param children The list of children INodes
      * @param children The list of children INodes
      * @param out The DataOutputStream to write
      * @param out The DataOutputStream to write
+     * @param inSnapshot Whether the parent directory or its ancestor is in 
+     *                   the deleted list of some snapshot (caused by rename or 
+     *                   deletion)
      * @param counter Counter to increment for namenode startup progress
      * @param counter Counter to increment for namenode startup progress
      * @return Number of children that are directory
      * @return Number of children that are directory
      */
      */
-    private int saveChildren(ReadOnlyList<INode> children, DataOutputStream out,
-        Counter counter) throws IOException {
+    private int saveChildren(ReadOnlyList<INode> children,
+        DataOutputStream out, boolean inSnapshot, Counter counter)
+        throws IOException {
       // Write normal children INode. 
       // Write normal children INode. 
       out.writeInt(children.size());
       out.writeInt(children.size());
       int dirNum = 0;
       int dirNum = 0;
       int i = 0;
       int i = 0;
       for(INode child : children) {
       for(INode child : children) {
         // print all children first
         // print all children first
+        // TODO: for HDFS-5428, we cannot change the format/content of fsimage
+        // here, thus even if the parent directory is in snapshot, we still
+        // do not handle INodeUC as those stored in deleted list
         saveINode2Image(child, out, false, referenceMap, counter);
         saveINode2Image(child, out, false, referenceMap, counter);
         if (child.isDirectory()) {
         if (child.isDirectory()) {
           dirNum++;
           dirNum++;
+        } else if (inSnapshot && child.isFile()
+            && child.asFile().isUnderConstruction()) {
+          this.snapshotUCMap.put(child.getId(),
+              (INodeFileUnderConstruction) child.asFile());
         }
         }
         if (i++ % 50 == 0) {
         if (i++ % 50 == 0) {
           context.checkCancelled();
           context.checkCancelled();
@@ -1054,14 +1113,15 @@ public class FSImageFormat {
      * 
      * 
      * @param current The current node
      * @param current The current node
      * @param out The DataoutputStream to write the image
      * @param out The DataoutputStream to write the image
-     * @param snapshot The possible snapshot associated with the current node
      * @param toSaveSubtree Whether or not to save the subtree to fsimage. For
      * @param toSaveSubtree Whether or not to save the subtree to fsimage. For
      *                      reference node, its subtree may already have been
      *                      reference node, its subtree may already have been
      *                      saved before.
      *                      saved before.
+     * @param inSnapshot Whether the current directory is in snapshot
      * @param counter Counter to increment for namenode startup progress
      * @param counter Counter to increment for namenode startup progress
      */
      */
     private void saveImage(INodeDirectory current, DataOutputStream out,
     private void saveImage(INodeDirectory current, DataOutputStream out,
-        boolean toSaveSubtree, Counter counter) throws IOException {
+        boolean toSaveSubtree, boolean inSnapshot, Counter counter)
+        throws IOException {
       // write the inode id of the directory
       // write the inode id of the directory
       out.writeLong(current.getId());
       out.writeLong(current.getId());
       
       
@@ -1090,7 +1150,7 @@ public class FSImageFormat {
       }
       }
 
 
       // 3. Write children INode 
       // 3. Write children INode 
-      dirNum += saveChildren(children, out, counter);
+      dirNum += saveChildren(children, out, inSnapshot, counter);
       
       
       // 4. Write DirectoryDiff lists, if there is any.
       // 4. Write DirectoryDiff lists, if there is any.
       SnapshotFSImageFormat.saveDirectoryDiffList(current, out, referenceMap);
       SnapshotFSImageFormat.saveDirectoryDiffList(current, out, referenceMap);
@@ -1105,14 +1165,14 @@ public class FSImageFormat {
         // make sure we only save the subtree under a reference node once
         // make sure we only save the subtree under a reference node once
         boolean toSave = child.isReference() ? 
         boolean toSave = child.isReference() ? 
             referenceMap.toProcessSubtree(child.getId()) : true;
             referenceMap.toProcessSubtree(child.getId()) : true;
-        saveImage(child.asDirectory(), out, toSave, counter);
+        saveImage(child.asDirectory(), out, toSave, inSnapshot, counter);
       }
       }
       if (snapshotDirs != null) {
       if (snapshotDirs != null) {
         for (INodeDirectory subDir : snapshotDirs) {
         for (INodeDirectory subDir : snapshotDirs) {
           // make sure we only save the subtree under a reference node once
           // make sure we only save the subtree under a reference node once
           boolean toSave = subDir.getParentReference() != null ? 
           boolean toSave = subDir.getParentReference() != null ? 
               referenceMap.toProcessSubtree(subDir.getId()) : true;
               referenceMap.toProcessSubtree(subDir.getId()) : true;
-          saveImage(subDir, out, toSave, counter);
+          saveImage(subDir, out, toSave, true, counter);
         }
         }
       }
       }
     }
     }

+ 23 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -5921,19 +5921,40 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
   /**
    * Serializes leases. 
    * Serializes leases. 
    */
    */
-  void saveFilesUnderConstruction(DataOutputStream out) throws IOException {
+  void saveFilesUnderConstruction(DataOutputStream out,
+      Map<Long, INodeFileUnderConstruction> snapshotUCMap) throws IOException {
     // This is run by an inferior thread of saveNamespace, which holds a read
     // This is run by an inferior thread of saveNamespace, which holds a read
     // lock on our behalf. If we took the read lock here, we could block
     // lock on our behalf. If we took the read lock here, we could block
     // for fairness if a writer is waiting on the lock.
     // for fairness if a writer is waiting on the lock.
     synchronized (leaseManager) {
     synchronized (leaseManager) {
       Map<String, INodeFileUnderConstruction> nodes =
       Map<String, INodeFileUnderConstruction> nodes =
           leaseManager.getINodesUnderConstruction();
           leaseManager.getINodesUnderConstruction();
-      out.writeInt(nodes.size()); // write the size    
+      for (Map.Entry<String, INodeFileUnderConstruction> entry
+          : nodes.entrySet()) {
+        // TODO: for HDFS-5428, because of rename operations, some
+        // under-construction files that are
+        // in the current fs directory can also be captured in the
+        // snapshotUCMap. We should remove them from the snapshotUCMap.
+        snapshotUCMap.remove(entry.getValue().getId());
+      }
+      
+      out.writeInt(nodes.size() + snapshotUCMap.size()); // write the size    
       for (Map.Entry<String, INodeFileUnderConstruction> entry
       for (Map.Entry<String, INodeFileUnderConstruction> entry
            : nodes.entrySet()) {
            : nodes.entrySet()) {
         FSImageSerialization.writeINodeUnderConstruction(
         FSImageSerialization.writeINodeUnderConstruction(
             out, entry.getValue(), entry.getKey());
             out, entry.getValue(), entry.getKey());
       }
       }
+      for (Map.Entry<Long, INodeFileUnderConstruction> entry
+          : snapshotUCMap.entrySet()) {
+        // for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>"
+        // as their paths
+        StringBuilder b = new StringBuilder();
+        b.append(FSDirectory.DOT_RESERVED_PATH_PREFIX)
+            .append(Path.SEPARATOR).append(FSDirectory.DOT_INODES_STRING)
+            .append(Path.SEPARATOR).append(entry.getValue().getId());
+        FSImageSerialization.writeINodeUnderConstruction(
+            out, entry.getValue(), b.toString());
+      }
     }
     }
   }
   }
 
 

+ 18 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -204,7 +204,24 @@ public class INodeDirectory extends INodeWithAdditionalFields
     clear();
     clear();
     return newDir;
     return newDir;
   }
   }
-
+  
+  /**
+   * Used when load fileUC from fsimage. The file to be replaced is actually 
+   * only in snapshot, thus may not be contained in the children list. 
+   * See HDFS-5428 for details.
+   */
+  public void replaceChildFileInSnapshot(INodeFile oldChild,
+      final INodeFile newChild) {
+    if (children != null) {
+      final int i = searchChildren(newChild.getLocalNameBytes());
+      if (i >= 0 && children.get(i).getId() == oldChild.getId()) {
+        // no need to consider reference node here, since we already do the 
+        // replacement in FSImageFormat.Loader#loadFilesUnderConstruction
+        children.set(i, newChild);
+      }
+    }
+  }
+  
   /** Replace the given child with a new child. */
   /** Replace the given child with a new child. */
   public void replaceChild(INode oldChild, final INode newChild,
   public void replaceChild(INode oldChild, final INode newChild,
       final INodeMap inodeMap) {
       final INodeMap inodeMap) {

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
@@ -75,7 +76,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
         final INode oldChild, final INode newChild) {
         final INode oldChild, final INode newChild) {
       final List<INode> list = getList(type); 
       final List<INode> list = getList(type); 
       final int i = search(list, oldChild.getLocalNameBytes());
       final int i = search(list, oldChild.getLocalNameBytes());
-      if (i < 0) {
+      if (i < 0 || list.get(i).getId() != oldChild.getId()) {
         return false;
         return false;
       }
       }
 
 
@@ -592,6 +593,14 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     return removed;
     return removed;
   }
   }
   
   
+  @Override
+  public void replaceChildFileInSnapshot(final INodeFile oldChild,
+      final INodeFile newChild) {
+    super.replaceChildFileInSnapshot(oldChild, newChild);
+    diffs.replaceChild(ListType.DELETED, oldChild, newChild);
+    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
+  }
+  
   @Override
   @Override
   public void replaceChild(final INode oldChild, final INode newChild,
   public void replaceChild(final INode oldChild, final INode newChild,
       final INodeMap inodeMap) {
       final INodeMap inodeMap) {

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -741,14 +741,15 @@ public class CacheAdmin extends Configured implements Tool {
         RemoteIterator<CachePoolInfo> iter = dfs.listCachePools();
         RemoteIterator<CachePoolInfo> iter = dfs.listCachePools();
         while (iter.hasNext()) {
         while (iter.hasNext()) {
           CachePoolInfo info = iter.next();
           CachePoolInfo info = iter.next();
+          String[] row = new String[5];
           if (name == null || info.getPoolName().equals(name)) {
           if (name == null || info.getPoolName().equals(name)) {
-            listing.addRow(new String[] {
-                info.getPoolName(),
-                info.getOwnerName(),
-                info.getGroupName(),
-                info.getMode().toString(),
-                info.getWeight().toString(),
-            });
+            row[0] = info.getPoolName();
+            row[1] = info.getOwnerName();
+            row[2] = info.getGroupName();
+            row[3] = info.getMode() != null ? info.getMode().toString() : null;
+            row[4] =
+                info.getWeight() != null ? info.getWeight().toString() : null;
+            listing.addRow(row);
             ++numResults;
             ++numResults;
             if (name != null) {
             if (name != null) {
               break;
               break;

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/TableListing.java

@@ -59,6 +59,9 @@ public class TableListing {
     }
     }
 
 
     private void addRow(String val) {
     private void addRow(String val) {
+      if (val == null) {
+        val = "";
+      }
       if ((val.length() + 1) > maxWidth) {
       if ((val.length() + 1) > maxWidth) {
         maxWidth = val.length() + 1;
         maxWidth = val.length() + 1;
       }
       }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -18,7 +18,7 @@
     "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
     "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <head>
 <head>
-<link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
+<link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
 <title>Namenode information</title>
 <title>Namenode information</title>
 </head>
 </head>
@@ -36,8 +36,8 @@
 <div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
 <div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
 </div>
 </div>
 
 
-<script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
-</script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
+<script type="text/javascript" src="/static/jquery-1.10.2.min.js">
+</script><script type="text/javascript" src="/static/bootstrap-3.0.2/js/bootstrap.min.js">
 </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
 </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
 </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
 </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
 </script><script type="text/javascript" src="dfs-dust.js">
 </script><script type="text/javascript" src="dfs-dust.js">

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -18,7 +18,7 @@
 	  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 	  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <html xmlns="http://www.w3.org/1999/xhtml">
   <head>
   <head>
-    <link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
+    <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />
     <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
     <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
     <title>Browsing HDFS</title>
     <title>Browsing HDFS</title>
   </head>
   </head>
@@ -73,8 +73,8 @@
       <br />
       <br />
       <div id="panel"></div>
       <div id="panel"></div>
     </div>
     </div>
-    <script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
-    </script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
+    <script type="text/javascript" src="/static/jquery-1.10.2.min.js">
+    </script><script type="text/javascript" src="/static/bootstrap-3.0.2/js/bootstrap.min.js">
     </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
     </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
     </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
     </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
     </script><script type="text/javascript" src="dfs-dust.js">
     </script><script type="text/javascript" src="dfs-dust.js">

File diff suppressed because it is too large
+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/css/bootstrap.min.css


BIN
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.eot


+ 229 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.svg

@@ -0,0 +1,229 @@
+<?xml version="1.0" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
+<svg xmlns="http://www.w3.org/2000/svg">
+<metadata></metadata>
+<defs>
+<font id="glyphicons_halflingsregular" horiz-adv-x="1200" >
+<font-face units-per-em="1200" ascent="960" descent="-240" />
+<missing-glyph horiz-adv-x="500" />
+<glyph />
+<glyph />
+<glyph unicode="&#xd;" />
+<glyph unicode=" " />
+<glyph unicode="*" d="M100 500v200h259l-183 183l141 141l183 -183v259h200v-259l183 183l141 -141l-183 -183h259v-200h-259l183 -183l-141 -141l-183 183v-259h-200v259l-183 -183l-141 141l183 183h-259z" />
+<glyph unicode="+" d="M0 400v300h400v400h300v-400h400v-300h-400v-400h-300v400h-400z" />
+<glyph unicode="&#xa0;" />
+<glyph unicode="&#x2000;" horiz-adv-x="652" />
+<glyph unicode="&#x2001;" horiz-adv-x="1304" />
+<glyph unicode="&#x2002;" horiz-adv-x="652" />
+<glyph unicode="&#x2003;" horiz-adv-x="1304" />
+<glyph unicode="&#x2004;" horiz-adv-x="434" />
+<glyph unicode="&#x2005;" horiz-adv-x="326" />
+<glyph unicode="&#x2006;" horiz-adv-x="217" />
+<glyph unicode="&#x2007;" horiz-adv-x="217" />
+<glyph unicode="&#x2008;" horiz-adv-x="163" />
+<glyph unicode="&#x2009;" horiz-adv-x="260" />
+<glyph unicode="&#x200a;" horiz-adv-x="72" />
+<glyph unicode="&#x202f;" horiz-adv-x="260" />
+<glyph unicode="&#x205f;" horiz-adv-x="326" />
+<glyph unicode="&#x20ac;" d="M100 500l100 100h113q0 47 5 100h-218l100 100h135q37 167 112 257q117 141 297 141q242 0 354 -189q60 -103 66 -209h-181q0 55 -25.5 99t-63.5 68t-75 36.5t-67 12.5q-24 0 -52.5 -10t-62.5 -32t-65.5 -67t-50.5 -107h379l-100 -100h-300q-6 -46 -6 -100h406l-100 -100 h-300q9 -74 33 -132t52.5 -91t62 -54.5t59 -29t46.5 -7.5q29 0 66 13t75 37t63.5 67.5t25.5 96.5h174q-31 -172 -128 -278q-107 -117 -274 -117q-205 0 -324 158q-36 46 -69 131.5t-45 205.5h-217z" />
+<glyph unicode="&#x2212;" d="M200 400h900v300h-900v-300z" />
+<glyph unicode="&#x2601;" d="M-14 494q0 -80 56.5 -137t135.5 -57h750q120 0 205 86t85 208q0 120 -85 206.5t-205 86.5q-46 0 -90 -14q-44 97 -134.5 156.5t-200.5 59.5q-152 0 -260 -107.5t-108 -260.5q0 -25 2 -37q-66 -14 -108.5 -67.5t-42.5 -122.5z" />
+<glyph unicode="&#x2709;" d="M0 100l400 400l200 -200l200 200l400 -400h-1200zM0 300v600l300 -300zM0 1100l600 -603l600 603h-1200zM900 600l300 300v-600z" />
+<glyph unicode="&#x270f;" d="M-13 -13l333 112l-223 223zM187 403l214 -214l614 614l-214 214zM887 1103l214 -214l99 92q13 13 13 32.5t-13 33.5l-153 153q-15 13 -33 13t-33 -13z" />
+<glyph unicode="&#xe000;" horiz-adv-x="500" d="M0 0z" />
+<glyph unicode="&#xe001;" d="M0 1200h1200l-500 -550v-550h300v-100h-800v100h300v550z" />
+<glyph unicode="&#xe002;" d="M14 84q18 -55 86 -75.5t147 5.5q65 21 109 69t44 90v606l600 155v-521q-64 16 -138 -7q-79 -26 -122.5 -83t-25.5 -111q17 -55 85.5 -75.5t147.5 4.5q70 23 111.5 63.5t41.5 95.5v881q0 10 -7 15.5t-17 2.5l-752 -193q-10 -3 -17 -12.5t-7 -19.5v-689q-64 17 -138 -7 q-79 -25 -122.5 -82t-25.5 -112z" />
+<glyph unicode="&#xe003;" d="M23 693q0 200 142 342t342 142t342 -142t142 -342q0 -142 -78 -261l300 -300q7 -8 7 -18t-7 -18l-109 -109q-8 -7 -18 -7t-18 7l-300 300q-119 -78 -261 -78q-200 0 -342 142t-142 342zM176 693q0 -136 97 -233t234 -97t233.5 96.5t96.5 233.5t-96.5 233.5t-233.5 96.5 t-234 -97t-97 -233z" />
+<glyph unicode="&#xe005;" d="M100 784q0 64 28 123t73 100.5t104.5 64t119 20.5t120 -38.5t104.5 -104.5q48 69 109.5 105t121.5 38t118.5 -20.5t102.5 -64t71 -100.5t27 -123q0 -57 -33.5 -117.5t-94 -124.5t-126.5 -127.5t-150 -152.5t-146 -174q-62 85 -145.5 174t-149.5 152.5t-126.5 127.5 t-94 124.5t-33.5 117.5z" />
+<glyph unicode="&#xe006;" d="M-72 800h479l146 400h2l146 -400h472l-382 -278l145 -449l-384 275l-382 -275l146 447zM168 71l2 1z" />
+<glyph unicode="&#xe007;" d="M-72 800h479l146 400h2l146 -400h472l-382 -278l145 -449l-384 275l-382 -275l146 447zM168 71l2 1zM237 700l196 -142l-73 -226l192 140l195 -141l-74 229l193 140h-235l-77 211l-78 -211h-239z" />
+<glyph unicode="&#xe008;" d="M0 0v143l400 257v100q-37 0 -68.5 74.5t-31.5 125.5v200q0 124 88 212t212 88t212 -88t88 -212v-200q0 -51 -31.5 -125.5t-68.5 -74.5v-100l400 -257v-143h-1200z" />
+<glyph unicode="&#xe009;" d="M0 0v1100h1200v-1100h-1200zM100 100h100v100h-100v-100zM100 300h100v100h-100v-100zM100 500h100v100h-100v-100zM100 700h100v100h-100v-100zM100 900h100v100h-100v-100zM300 100h600v400h-600v-400zM300 600h600v400h-600v-400zM1000 100h100v100h-100v-100z M1000 300h100v100h-100v-100zM1000 500h100v100h-100v-100zM1000 700h100v100h-100v-100zM1000 900h100v100h-100v-100z" />
+<glyph unicode="&#xe010;" d="M0 50v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-400q-21 0 -35.5 14.5t-14.5 35.5zM0 650v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-400 q-21 0 -35.5 14.5t-14.5 35.5zM600 50v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-400q-21 0 -35.5 14.5t-14.5 35.5zM600 650v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400 q0 -21 -14.5 -35.5t-35.5 -14.5h-400q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe011;" d="M0 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM0 450v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200 q-21 0 -35.5 14.5t-14.5 35.5zM0 850v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM400 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5 t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM400 450v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM400 850v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5 v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM800 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM800 450v200q0 21 14.5 35.5t35.5 14.5h200 q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM800 850v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe012;" d="M0 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM0 450q0 -21 14.5 -35.5t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v200q0 21 -14.5 35.5t-35.5 14.5h-200q-21 0 -35.5 -14.5 t-14.5 -35.5v-200zM0 850v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM400 50v200q0 21 14.5 35.5t35.5 14.5h700q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5 t-35.5 -14.5h-700q-21 0 -35.5 14.5t-14.5 35.5zM400 450v200q0 21 14.5 35.5t35.5 14.5h700q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-700q-21 0 -35.5 14.5t-14.5 35.5zM400 850v200q0 21 14.5 35.5t35.5 14.5h700q21 0 35.5 -14.5t14.5 -35.5 v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-700q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe013;" d="M29 454l419 -420l818 820l-212 212l-607 -607l-206 207z" />
+<glyph unicode="&#xe014;" d="M106 318l282 282l-282 282l212 212l282 -282l282 282l212 -212l-282 -282l282 -282l-212 -212l-282 282l-282 -282z" />
+<glyph unicode="&#xe015;" d="M23 693q0 200 142 342t342 142t342 -142t142 -342q0 -142 -78 -261l300 -300q7 -8 7 -18t-7 -18l-109 -109q-8 -7 -18 -7t-18 7l-300 300q-119 -78 -261 -78q-200 0 -342 142t-142 342zM176 693q0 -136 97 -233t234 -97t233.5 96.5t96.5 233.5t-96.5 233.5t-233.5 96.5 t-234 -97t-97 -233zM300 600v200h100v100h200v-100h100v-200h-100v-100h-200v100h-100z" />
+<glyph unicode="&#xe016;" d="M23 694q0 200 142 342t342 142t342 -142t142 -342q0 -141 -78 -262l300 -299q7 -7 7 -18t-7 -18l-109 -109q-8 -8 -18 -8t-18 8l-300 299q-120 -77 -261 -77q-200 0 -342 142t-142 342zM176 694q0 -136 97 -233t234 -97t233.5 97t96.5 233t-96.5 233t-233.5 97t-234 -97 t-97 -233zM300 601h400v200h-400v-200z" />
+<glyph unicode="&#xe017;" d="M23 600q0 183 105 331t272 210v-166q-103 -55 -165 -155t-62 -220q0 -177 125 -302t302 -125t302 125t125 302q0 120 -62 220t-165 155v166q167 -62 272 -210t105 -331q0 -118 -45.5 -224.5t-123 -184t-184 -123t-224.5 -45.5t-224.5 45.5t-184 123t-123 184t-45.5 224.5 zM500 750q0 -21 14.5 -35.5t35.5 -14.5h100q21 0 35.5 14.5t14.5 35.5v400q0 21 -14.5 35.5t-35.5 14.5h-100q-21 0 -35.5 -14.5t-14.5 -35.5v-400z" />
+<glyph unicode="&#xe018;" d="M100 1h200v300h-200v-300zM400 1v500h200v-500h-200zM700 1v800h200v-800h-200zM1000 1v1200h200v-1200h-200z" />
+<glyph unicode="&#xe019;" d="M26 601q0 -33 6 -74l151 -38l2 -6q14 -49 38 -93l3 -5l-80 -134q45 -59 105 -105l133 81l5 -3q45 -26 94 -39l5 -2l38 -151q40 -5 74 -5q27 0 74 5l38 151l6 2q46 13 93 39l5 3l134 -81q56 44 104 105l-80 134l3 5q24 44 39 93l1 6l152 38q5 40 5 74q0 28 -5 73l-152 38 l-1 6q-16 51 -39 93l-3 5l80 134q-44 58 -104 105l-134 -81l-5 3q-45 25 -93 39l-6 1l-38 152q-40 5 -74 5q-27 0 -74 -5l-38 -152l-5 -1q-50 -14 -94 -39l-5 -3l-133 81q-59 -47 -105 -105l80 -134l-3 -5q-25 -47 -38 -93l-2 -6l-151 -38q-6 -48 -6 -73zM385 601 q0 88 63 151t152 63t152 -63t63 -151q0 -89 -63 -152t-152 -63t-152 63t-63 152z" />
+<glyph unicode="&#xe020;" d="M100 1025v50q0 10 7.5 17.5t17.5 7.5h275v100q0 41 29.5 70.5t70.5 29.5h300q41 0 70.5 -29.5t29.5 -70.5v-100h275q10 0 17.5 -7.5t7.5 -17.5v-50q0 -11 -7 -18t-18 -7h-1050q-11 0 -18 7t-7 18zM200 100v800h900v-800q0 -41 -29.5 -71t-70.5 -30h-700q-41 0 -70.5 30 t-29.5 71zM300 100h100v700h-100v-700zM500 100h100v700h-100v-700zM500 1100h300v100h-300v-100zM700 100h100v700h-100v-700zM900 100h100v700h-100v-700z" />
+<glyph unicode="&#xe021;" d="M1 601l656 644l644 -644h-200v-600h-300v400h-300v-400h-300v600h-200z" />
+<glyph unicode="&#xe022;" d="M100 25v1150q0 11 7 18t18 7h475v-500h400v-675q0 -11 -7 -18t-18 -7h-850q-11 0 -18 7t-7 18zM700 800v300l300 -300h-300z" />
+<glyph unicode="&#xe023;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM500 500v400h100 v-300h200v-100h-300z" />
+<glyph unicode="&#xe024;" d="M-100 0l431 1200h209l-21 -300h162l-20 300h208l431 -1200h-538l-41 400h-242l-40 -400h-539zM488 500h224l-27 300h-170z" />
+<glyph unicode="&#xe025;" d="M0 0v400h490l-290 300h200v500h300v-500h200l-290 -300h490v-400h-1100zM813 200h175v100h-175v-100z" />
+<glyph unicode="&#xe026;" d="M1 600q0 122 47.5 233t127.5 191t191 127.5t233 47.5t233 -47.5t191 -127.5t127.5 -191t47.5 -233t-47.5 -233t-127.5 -191t-191 -127.5t-233 -47.5t-233 47.5t-191 127.5t-127.5 191t-47.5 233zM188 600q0 -170 121 -291t291 -121t291 121t121 291t-121 291t-291 121 t-291 -121t-121 -291zM350 600h150v300h200v-300h150l-250 -300z" />
+<glyph unicode="&#xe027;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM350 600l250 300 l250 -300h-150v-300h-200v300h-150z" />
+<glyph unicode="&#xe028;" d="M0 25v475l200 700h800q199 -700 200 -700v-475q0 -11 -7 -18t-18 -7h-1150q-11 0 -18 7t-7 18zM200 500h200l50 -200h300l50 200h200l-97 500h-606z" />
+<glyph unicode="&#xe029;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -172 121.5 -293t292.5 -121t292.5 121t121.5 293q0 171 -121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM500 397v401 l297 -200z" />
+<glyph unicode="&#xe030;" d="M23 600q0 -118 45.5 -224.5t123 -184t184 -123t224.5 -45.5t224.5 45.5t184 123t123 184t45.5 224.5h-150q0 -177 -125 -302t-302 -125t-302 125t-125 302t125 302t302 125q136 0 246 -81l-146 -146h400v400l-145 -145q-157 122 -355 122q-118 0 -224.5 -45.5t-184 -123 t-123 -184t-45.5 -224.5z" />
+<glyph unicode="&#xe031;" d="M23 600q0 118 45.5 224.5t123 184t184 123t224.5 45.5q198 0 355 -122l145 145v-400h-400l147 147q-112 80 -247 80q-177 0 -302 -125t-125 -302h-150zM100 0v400h400l-147 -147q112 -80 247 -80q177 0 302 125t125 302h150q0 -118 -45.5 -224.5t-123 -184t-184 -123 t-224.5 -45.5q-198 0 -355 122z" />
+<glyph unicode="&#xe032;" d="M100 0h1100v1200h-1100v-1200zM200 100v900h900v-900h-900zM300 200v100h100v-100h-100zM300 400v100h100v-100h-100zM300 600v100h100v-100h-100zM300 800v100h100v-100h-100zM500 200h500v100h-500v-100zM500 400v100h500v-100h-500zM500 600v100h500v-100h-500z M500 800v100h500v-100h-500z" />
+<glyph unicode="&#xe033;" d="M0 100v600q0 41 29.5 70.5t70.5 29.5h100v200q0 82 59 141t141 59h300q82 0 141 -59t59 -141v-200h100q41 0 70.5 -29.5t29.5 -70.5v-600q0 -41 -29.5 -70.5t-70.5 -29.5h-900q-41 0 -70.5 29.5t-29.5 70.5zM400 800h300v150q0 21 -14.5 35.5t-35.5 14.5h-200 q-21 0 -35.5 -14.5t-14.5 -35.5v-150z" />
+<glyph unicode="&#xe034;" d="M100 0v1100h100v-1100h-100zM300 400q60 60 127.5 84t127.5 17.5t122 -23t119 -30t110 -11t103 42t91 120.5v500q-40 -81 -101.5 -115.5t-127.5 -29.5t-138 25t-139.5 40t-125.5 25t-103 -29.5t-65 -115.5v-500z" />
+<glyph unicode="&#xe035;" d="M0 275q0 -11 7 -18t18 -7h50q11 0 18 7t7 18v300q0 127 70.5 231.5t184.5 161.5t245 57t245 -57t184.5 -161.5t70.5 -231.5v-300q0 -11 7 -18t18 -7h50q11 0 18 7t7 18v300q0 116 -49.5 227t-131 192.5t-192.5 131t-227 49.5t-227 -49.5t-192.5 -131t-131 -192.5 t-49.5 -227v-300zM200 20v460q0 8 6 14t14 6h160q8 0 14 -6t6 -14v-460q0 -8 -6 -14t-14 -6h-160q-8 0 -14 6t-6 14zM800 20v460q0 8 6 14t14 6h160q8 0 14 -6t6 -14v-460q0 -8 -6 -14t-14 -6h-160q-8 0 -14 6t-6 14z" />
+<glyph unicode="&#xe036;" d="M0 400h300l300 -200v800l-300 -200h-300v-400zM688 459l141 141l-141 141l71 71l141 -141l141 141l71 -71l-141 -141l141 -141l-71 -71l-141 141l-141 -141z" />
+<glyph unicode="&#xe037;" d="M0 400h300l300 -200v800l-300 -200h-300v-400zM700 857l69 53q111 -135 111 -310q0 -169 -106 -302l-67 54q86 110 86 248q0 146 -93 257z" />
+<glyph unicode="&#xe038;" d="M0 401v400h300l300 200v-800l-300 200h-300zM702 858l69 53q111 -135 111 -310q0 -170 -106 -303l-67 55q86 110 86 248q0 145 -93 257zM889 951l7 -8q123 -151 123 -344q0 -189 -119 -339l-7 -8l81 -66l6 8q142 178 142 405q0 230 -144 408l-6 8z" />
+<glyph unicode="&#xe039;" d="M0 0h500v500h-200v100h-100v-100h-200v-500zM0 600h100v100h400v100h100v100h-100v300h-500v-600zM100 100v300h300v-300h-300zM100 800v300h300v-300h-300zM200 200v100h100v-100h-100zM200 900h100v100h-100v-100zM500 500v100h300v-300h200v-100h-100v-100h-200v100 h-100v100h100v200h-200zM600 0v100h100v-100h-100zM600 1000h100v-300h200v-300h300v200h-200v100h200v500h-600v-200zM800 800v300h300v-300h-300zM900 0v100h300v-100h-300zM900 900v100h100v-100h-100zM1100 200v100h100v-100h-100z" />
+<glyph unicode="&#xe040;" d="M0 200h100v1000h-100v-1000zM100 0v100h300v-100h-300zM200 200v1000h100v-1000h-100zM500 0v91h100v-91h-100zM500 200v1000h200v-1000h-200zM700 0v91h100v-91h-100zM800 200v1000h100v-1000h-100zM900 0v91h200v-91h-200zM1000 200v1000h200v-1000h-200z" />
+<glyph unicode="&#xe041;" d="M1 700v475q0 10 7.5 17.5t17.5 7.5h474l700 -700l-500 -500zM148 953q0 -42 29 -71q30 -30 71.5 -30t71.5 30q29 29 29 71t-29 71q-30 30 -71.5 30t-71.5 -30q-29 -29 -29 -71z" />
+<glyph unicode="&#xe042;" d="M2 700v475q0 11 7 18t18 7h474l700 -700l-500 -500zM148 953q0 -42 30 -71q29 -30 71 -30t71 30q30 29 30 71t-30 71q-29 30 -71 30t-71 -30q-30 -29 -30 -71zM701 1200h100l700 -700l-500 -500l-50 50l450 450z" />
+<glyph unicode="&#xe043;" d="M100 0v1025l175 175h925v-1000l-100 -100v1000h-750l-100 -100h750v-1000h-900z" />
+<glyph unicode="&#xe044;" d="M200 0l450 444l450 -443v1150q0 20 -14.5 35t-35.5 15h-800q-21 0 -35.5 -15t-14.5 -35v-1151z" />
+<glyph unicode="&#xe045;" d="M0 100v700h200l100 -200h600l100 200h200v-700h-200v200h-800v-200h-200zM253 829l40 -124h592l62 124l-94 346q-2 11 -10 18t-18 7h-450q-10 0 -18 -7t-10 -18zM281 24l38 152q2 10 11.5 17t19.5 7h500q10 0 19.5 -7t11.5 -17l38 -152q2 -10 -3.5 -17t-15.5 -7h-600 q-10 0 -15.5 7t-3.5 17z" />
+<glyph unicode="&#xe046;" d="M0 200q0 -41 29.5 -70.5t70.5 -29.5h1000q41 0 70.5 29.5t29.5 70.5v600q0 41 -29.5 70.5t-70.5 29.5h-150q-4 8 -11.5 21.5t-33 48t-53 61t-69 48t-83.5 21.5h-200q-41 0 -82 -20.5t-70 -50t-52 -59t-34 -50.5l-12 -20h-150q-41 0 -70.5 -29.5t-29.5 -70.5v-600z M356 500q0 100 72 172t172 72t172 -72t72 -172t-72 -172t-172 -72t-172 72t-72 172zM494 500q0 -44 31 -75t75 -31t75 31t31 75t-31 75t-75 31t-75 -31t-31 -75zM900 700v100h100v-100h-100z" />
+<glyph unicode="&#xe047;" d="M53 0h365v66q-41 0 -72 11t-49 38t1 71l92 234h391l82 -222q16 -45 -5.5 -88.5t-74.5 -43.5v-66h417v66q-34 1 -74 43q-18 19 -33 42t-21 37l-6 13l-385 998h-93l-399 -1006q-24 -48 -52 -75q-12 -12 -33 -25t-36 -20l-15 -7v-66zM416 521l178 457l46 -140l116 -317h-340 z" />
+<glyph unicode="&#xe048;" d="M100 0v89q41 7 70.5 32.5t29.5 65.5v827q0 28 -1 39.5t-5.5 26t-15.5 21t-29 14t-49 14.5v70h471q120 0 213 -88t93 -228q0 -55 -11.5 -101.5t-28 -74t-33.5 -47.5t-28 -28l-12 -7q8 -3 21.5 -9t48 -31.5t60.5 -58t47.5 -91.5t21.5 -129q0 -84 -59 -156.5t-142 -111 t-162 -38.5h-500zM400 200h161q89 0 153 48.5t64 132.5q0 90 -62.5 154.5t-156.5 64.5h-159v-400zM400 700h139q76 0 130 61.5t54 138.5q0 82 -84 130.5t-239 48.5v-379z" />
+<glyph unicode="&#xe049;" d="M200 0v57q77 7 134.5 40.5t65.5 80.5l173 849q10 56 -10 74t-91 37q-6 1 -10.5 2.5t-9.5 2.5v57h425l2 -57q-33 -8 -62 -25.5t-46 -37t-29.5 -38t-17.5 -30.5l-5 -12l-128 -825q-10 -52 14 -82t95 -36v-57h-500z" />
+<glyph unicode="&#xe050;" d="M-75 200h75v800h-75l125 167l125 -167h-75v-800h75l-125 -167zM300 900v300h150h700h150v-300h-50q0 29 -8 48.5t-18.5 30t-33.5 15t-39.5 5.5t-50.5 1h-200v-850l100 -50v-100h-400v100l100 50v850h-200q-34 0 -50.5 -1t-40 -5.5t-33.5 -15t-18.5 -30t-8.5 -48.5h-49z " />
+<glyph unicode="&#xe051;" d="M33 51l167 125v-75h800v75l167 -125l-167 -125v75h-800v-75zM100 901v300h150h700h150v-300h-50q0 29 -8 48.5t-18 30t-33.5 15t-40 5.5t-50.5 1h-200v-650l100 -50v-100h-400v100l100 50v650h-200q-34 0 -50.5 -1t-39.5 -5.5t-33.5 -15t-18.5 -30t-8 -48.5h-50z" />
+<glyph unicode="&#xe052;" d="M0 50q0 -20 14.5 -35t35.5 -15h1100q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM0 350q0 -20 14.5 -35t35.5 -15h800q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-800q-21 0 -35.5 -14.5t-14.5 -35.5 v-100zM0 650q0 -20 14.5 -35t35.5 -15h1000q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1000q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM0 950q0 -20 14.5 -35t35.5 -15h600q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-600q-21 0 -35.5 -14.5 t-14.5 -35.5v-100z" />
+<glyph unicode="&#xe053;" d="M0 50q0 -20 14.5 -35t35.5 -15h1100q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM0 650q0 -20 14.5 -35t35.5 -15h1100q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5 v-100zM200 350q0 -20 14.5 -35t35.5 -15h700q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-700q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM200 950q0 -20 14.5 -35t35.5 -15h700q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-700q-21 0 -35.5 -14.5 t-14.5 -35.5v-100z" />
+<glyph unicode="&#xe054;" d="M0 50v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15t-14.5 35zM100 650v100q0 21 14.5 35.5t35.5 14.5h1000q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1000q-21 0 -35.5 15 t-14.5 35zM300 350v100q0 21 14.5 35.5t35.5 14.5h800q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-800q-21 0 -35.5 15t-14.5 35zM500 950v100q0 21 14.5 35.5t35.5 14.5h600q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-600 q-21 0 -35.5 15t-14.5 35z" />
+<glyph unicode="&#xe055;" d="M0 50v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15t-14.5 35zM0 350v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15 t-14.5 35zM0 650v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15t-14.5 35zM0 950v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100 q-21 0 -35.5 15t-14.5 35z" />
+<glyph unicode="&#xe056;" d="M0 50v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15t-14.5 35zM0 350v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15 t-14.5 35zM0 650v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15t-14.5 35zM0 950v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15 t-14.5 35zM300 50v100q0 21 14.5 35.5t35.5 14.5h800q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-800q-21 0 -35.5 15t-14.5 35zM300 350v100q0 21 14.5 35.5t35.5 14.5h800q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-800 q-21 0 -35.5 15t-14.5 35zM300 650v100q0 21 14.5 35.5t35.5 14.5h800q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-800q-21 0 -35.5 15t-14.5 35zM300 950v100q0 21 14.5 35.5t35.5 14.5h800q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15 h-800q-21 0 -35.5 15t-14.5 35z" />
+<glyph unicode="&#xe057;" d="M-101 500v100h201v75l166 -125l-166 -125v75h-201zM300 0h100v1100h-100v-1100zM500 50q0 -20 14.5 -35t35.5 -15h600q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-600q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM500 350q0 -20 14.5 -35t35.5 -15h300q20 0 35 15t15 35 v100q0 21 -15 35.5t-35 14.5h-300q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM500 650q0 -20 14.5 -35t35.5 -15h500q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-500q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM500 950q0 -20 14.5 -35t35.5 -15h100q20 0 35 15t15 35v100 q0 21 -15 35.5t-35 14.5h-100q-21 0 -35.5 -14.5t-14.5 -35.5v-100z" />
+<glyph unicode="&#xe058;" d="M1 50q0 -20 14.5 -35t35.5 -15h600q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-600q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM1 350q0 -20 14.5 -35t35.5 -15h300q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-300q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM1 650 q0 -20 14.5 -35t35.5 -15h500q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-500q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM1 950q0 -20 14.5 -35t35.5 -15h100q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-100q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM801 0v1100h100v-1100 h-100zM934 550l167 -125v75h200v100h-200v75z" />
+<glyph unicode="&#xe059;" d="M0 275v650q0 31 22 53t53 22h750q31 0 53 -22t22 -53v-650q0 -31 -22 -53t-53 -22h-750q-31 0 -53 22t-22 53zM900 600l300 300v-600z" />
+<glyph unicode="&#xe060;" d="M0 44v1012q0 18 13 31t31 13h1112q19 0 31.5 -13t12.5 -31v-1012q0 -18 -12.5 -31t-31.5 -13h-1112q-18 0 -31 13t-13 31zM100 263l247 182l298 -131l-74 156l293 318l236 -288v500h-1000v-737zM208 750q0 56 39 95t95 39t95 -39t39 -95t-39 -95t-95 -39t-95 39t-39 95z " />
+<glyph unicode="&#xe062;" d="M148 745q0 124 60.5 231.5t165 172t226.5 64.5q123 0 227 -63t164.5 -169.5t60.5 -229.5t-73 -272q-73 -114 -166.5 -237t-150.5 -189l-57 -66q-10 9 -27 26t-66.5 70.5t-96 109t-104 135.5t-100.5 155q-63 139 -63 262zM342 772q0 -107 75.5 -182.5t181.5 -75.5 q107 0 182.5 75.5t75.5 182.5t-75.5 182t-182.5 75t-182 -75.5t-75 -181.5z" />
+<glyph unicode="&#xe063;" d="M1 600q0 122 47.5 233t127.5 191t191 127.5t233 47.5t233 -47.5t191 -127.5t127.5 -191t47.5 -233t-47.5 -233t-127.5 -191t-191 -127.5t-233 -47.5t-233 47.5t-191 127.5t-127.5 191t-47.5 233zM173 600q0 -177 125.5 -302t301.5 -125v854q-176 0 -301.5 -125 t-125.5 -302z" />
+<glyph unicode="&#xe064;" d="M117 406q0 94 34 186t88.5 172.5t112 159t115 177t87.5 194.5q21 -71 57.5 -142.5t76 -130.5t83 -118.5t82 -117t70 -116t50 -125.5t18.5 -136q0 -89 -39 -165.5t-102 -126.5t-140 -79.5t-156 -33.5q-114 6 -211.5 53t-161.5 138.5t-64 210.5zM243 414q14 -82 59.5 -136 t136.5 -80l16 98q-7 6 -18 17t-34 48t-33 77q-15 73 -14 143.5t10 122.5l9 51q-92 -110 -119.5 -185t-12.5 -156z" />
+<glyph unicode="&#xe065;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5q366 -6 397 -14l-186 -186h-311q-41 0 -70.5 -29.5t-29.5 -70.5v-500q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v125l200 200v-225q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5 t-117.5 282.5zM436 341l161 50l412 412l-114 113l-405 -405zM995 1015l113 -113l113 113l-21 85l-92 28z" />
+<glyph unicode="&#xe066;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h261l2 -80q-133 -32 -218 -120h-145q-41 0 -70.5 -29.5t-29.5 -70.5v-500q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5l200 153v-53q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5t-117.5 282.5 zM423 524q30 38 81.5 64t103 35.5t99 14t77.5 3.5l29 -1v-209l360 324l-359 318v-216q-7 0 -19 -1t-48 -8t-69.5 -18.5t-76.5 -37t-76.5 -59t-62 -88t-39.5 -121.5z" />
+<glyph unicode="&#xe067;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h300q60 0 127 -23l-178 -177h-349q-41 0 -70.5 -29.5t-29.5 -70.5v-500q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v69l200 200v-169q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5 t-117.5 282.5zM342 632l283 -284l566 567l-136 137l-430 -431l-147 147z" />
+<glyph unicode="&#xe068;" d="M0 603l300 296v-198h200v200h-200l300 300l295 -300h-195v-200h200v198l300 -296l-300 -300v198h-200v-200h195l-295 -300l-300 300h200v200h-200v-198z" />
+<glyph unicode="&#xe069;" d="M200 50v1000q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-437l500 487v-1100l-500 488v-438q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe070;" d="M0 50v1000q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-437l500 487v-487l500 487v-1100l-500 488v-488l-500 488v-438q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe071;" d="M136 550l564 550v-487l500 487v-1100l-500 488v-488z" />
+<glyph unicode="&#xe072;" d="M200 0l900 550l-900 550v-1100z" />
+<glyph unicode="&#xe073;" d="M200 150q0 -21 14.5 -35.5t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v800q0 21 -14.5 35.5t-35.5 14.5h-200q-21 0 -35.5 -14.5t-14.5 -35.5v-800zM600 150q0 -21 14.5 -35.5t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v800q0 21 -14.5 35.5t-35.5 14.5h-200 q-21 0 -35.5 -14.5t-14.5 -35.5v-800z" />
+<glyph unicode="&#xe074;" d="M200 150q0 -20 14.5 -35t35.5 -15h800q21 0 35.5 15t14.5 35v800q0 21 -14.5 35.5t-35.5 14.5h-800q-21 0 -35.5 -14.5t-14.5 -35.5v-800z" />
+<glyph unicode="&#xe075;" d="M0 0v1100l500 -487v487l564 -550l-564 -550v488z" />
+<glyph unicode="&#xe076;" d="M0 0v1100l500 -487v487l500 -487v437q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-1000q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5v438l-500 -488v488z" />
+<glyph unicode="&#xe077;" d="M300 0v1100l500 -487v437q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-1000q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5v438z" />
+<glyph unicode="&#xe078;" d="M100 250v100q0 21 14.5 35.5t35.5 14.5h1000q21 0 35.5 -14.5t14.5 -35.5v-100q0 -21 -14.5 -35.5t-35.5 -14.5h-1000q-21 0 -35.5 14.5t-14.5 35.5zM100 500h1100l-550 564z" />
+<glyph unicode="&#xe079;" d="M185 599l592 -592l240 240l-353 353l353 353l-240 240z" />
+<glyph unicode="&#xe080;" d="M272 194l353 353l-353 353l241 240l572 -571l21 -22l-1 -1v-1l-592 -591z" />
+<glyph unicode="&#xe081;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -300t-217.5 -218t-299.5 -80t-299.5 80t-217.5 218t-80 300zM300 500h200v-200h200v200h200v200h-200v200h-200v-200h-200v-200z" />
+<glyph unicode="&#xe082;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -300t-217.5 -218t-299.5 -80t-299.5 80t-217.5 218t-80 300zM300 500h600v200h-600v-200z" />
+<glyph unicode="&#xe083;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -300t-217.5 -218t-299.5 -80t-299.5 80t-217.5 218t-80 300zM246 459l213 -213l141 142l141 -142l213 213l-142 141l142 141l-213 212l-141 -141l-141 142l-212 -213l141 -141z" />
+<glyph unicode="&#xe084;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM270 551l276 -277l411 411l-175 174l-236 -236l-102 102z" />
+<glyph unicode="&#xe085;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -300t-217.5 -218t-299.5 -80t-299.5 80t-217.5 218t-80 300zM363 700h144q4 0 11.5 -1t11 -1t6.5 3t3 9t1 11t3.5 8.5t3.5 6t5.5 4t6.5 2.5t9 1.5t9 0.5h11.5h12.5q19 0 30 -10t11 -26 q0 -22 -4 -28t-27 -22q-5 -1 -12.5 -3t-27 -13.5t-34 -27t-26.5 -46t-11 -68.5h200q5 3 14 8t31.5 25.5t39.5 45.5t31 69t14 94q0 51 -17.5 89t-42 58t-58.5 32t-58.5 15t-51.5 3q-105 0 -172 -56t-67 -183zM500 300h200v100h-200v-100z" />
+<glyph unicode="&#xe086;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -300t-217.5 -218t-299.5 -80t-299.5 80t-217.5 218t-80 300zM400 300h400v100h-100v300h-300v-100h100v-200h-100v-100zM500 800h200v100h-200v-100z" />
+<glyph unicode="&#xe087;" d="M0 500v200h194q15 60 36 104.5t55.5 86t88 69t126.5 40.5v200h200v-200q54 -20 113 -60t112.5 -105.5t71.5 -134.5h203v-200h-203q-25 -102 -116.5 -186t-180.5 -117v-197h-200v197q-140 27 -208 102.5t-98 200.5h-194zM290 500q24 -73 79.5 -127.5t130.5 -78.5v206h200 v-206q149 48 201 206h-201v200h200q-25 74 -76 127.5t-124 76.5v-204h-200v203q-75 -24 -130 -77.5t-79 -125.5h209v-200h-210z" />
+<glyph unicode="&#xe088;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM356 465l135 135 l-135 135l109 109l135 -135l135 135l109 -109l-135 -135l135 -135l-109 -109l-135 135l-135 -135z" />
+<glyph unicode="&#xe089;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM322 537l141 141 l87 -87l204 205l142 -142l-346 -345z" />
+<glyph unicode="&#xe090;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -115 62 -215l568 567q-100 62 -216 62q-171 0 -292.5 -121.5t-121.5 -292.5zM391 245q97 -59 209 -59q171 0 292.5 121.5t121.5 292.5 q0 112 -59 209z" />
+<glyph unicode="&#xe091;" d="M0 547l600 453v-300h600v-300h-600v-301z" />
+<glyph unicode="&#xe092;" d="M0 400v300h600v300l600 -453l-600 -448v301h-600z" />
+<glyph unicode="&#xe093;" d="M204 600l450 600l444 -600h-298v-600h-300v600h-296z" />
+<glyph unicode="&#xe094;" d="M104 600h296v600h300v-600h298l-449 -600z" />
+<glyph unicode="&#xe095;" d="M0 200q6 132 41 238.5t103.5 193t184 138t271.5 59.5v271l600 -453l-600 -448v301q-95 -2 -183 -20t-170 -52t-147 -92.5t-100 -135.5z" />
+<glyph unicode="&#xe096;" d="M0 0v400l129 -129l294 294l142 -142l-294 -294l129 -129h-400zM635 777l142 -142l294 294l129 -129v400h-400l129 -129z" />
+<glyph unicode="&#xe097;" d="M34 176l295 295l-129 129h400v-400l-129 130l-295 -295zM600 600v400l129 -129l295 295l142 -141l-295 -295l129 -130h-400z" />
+<glyph unicode="&#xe101;" d="M23 600q0 118 45.5 224.5t123 184t184 123t224.5 45.5t224.5 -45.5t184 -123t123 -184t45.5 -224.5t-45.5 -224.5t-123 -184t-184 -123t-224.5 -45.5t-224.5 45.5t-184 123t-123 184t-45.5 224.5zM456 851l58 -302q4 -20 21.5 -34.5t37.5 -14.5h54q20 0 37.5 14.5 t21.5 34.5l58 302q4 20 -8 34.5t-33 14.5h-207q-20 0 -32 -14.5t-8 -34.5zM500 300h200v100h-200v-100z" />
+<glyph unicode="&#xe102;" d="M0 800h100v-200h400v300h200v-300h400v200h100v100h-111v6t-1 15t-3 18l-34 172q-11 39 -41.5 63t-69.5 24q-32 0 -61 -17l-239 -144q-22 -13 -40 -35q-19 24 -40 36l-238 144q-33 18 -62 18q-39 0 -69.5 -23t-40.5 -61l-35 -177q-2 -8 -3 -18t-1 -15v-6h-111v-100z M100 0h400v400h-400v-400zM200 900q-3 0 14 48t35 96l18 47l214 -191h-281zM700 0v400h400v-400h-400zM731 900l202 197q5 -12 12 -32.5t23 -64t25 -72t7 -28.5h-269z" />
+<glyph unicode="&#xe103;" d="M0 -22v143l216 193q-9 53 -13 83t-5.5 94t9 113t38.5 114t74 124q47 60 99.5 102.5t103 68t127.5 48t145.5 37.5t184.5 43.5t220 58.5q0 -189 -22 -343t-59 -258t-89 -181.5t-108.5 -120t-122 -68t-125.5 -30t-121.5 -1.5t-107.5 12.5t-87.5 17t-56.5 7.5l-99 -55z M238.5 300.5q19.5 -6.5 86.5 76.5q55 66 367 234q70 38 118.5 69.5t102 79t99 111.5t86.5 148q22 50 24 60t-6 19q-7 5 -17 5t-26.5 -14.5t-33.5 -39.5q-35 -51 -113.5 -108.5t-139.5 -89.5l-61 -32q-369 -197 -458 -401q-48 -111 -28.5 -117.5z" />
+<glyph unicode="&#xe104;" d="M111 408q0 -33 5 -63q9 -56 44 -119.5t105 -108.5q31 -21 64 -16t62 23.5t57 49.5t48 61.5t35 60.5q32 66 39 184.5t-13 157.5q79 -80 122 -164t26 -184q-5 -33 -20.5 -69.5t-37.5 -80.5q-10 -19 -14.5 -29t-12 -26t-9 -23.5t-3 -19t2.5 -15.5t11 -9.5t19.5 -5t30.5 2.5 t42 8q57 20 91 34t87.5 44.5t87 64t65.5 88.5t47 122q38 172 -44.5 341.5t-246.5 278.5q22 -44 43 -129q39 -159 -32 -154q-15 2 -33 9q-79 33 -120.5 100t-44 175.5t48.5 257.5q-13 -8 -34 -23.5t-72.5 -66.5t-88.5 -105.5t-60 -138t-8 -166.5q2 -12 8 -41.5t8 -43t6 -39.5 t3.5 -39.5t-1 -33.5t-6 -31.5t-13.5 -24t-21 -20.5t-31 -12q-38 -10 -67 13t-40.5 61.5t-15 81.5t10.5 75q-52 -46 -83.5 -101t-39 -107t-7.5 -85z" />
+<glyph unicode="&#xe105;" d="M-61 600l26 40q6 10 20 30t49 63.5t74.5 85.5t97 90t116.5 83.5t132.5 59t145.5 23.5t145.5 -23.5t132.5 -59t116.5 -83.5t97 -90t74.5 -85.5t49 -63.5t20 -30l26 -40l-26 -40q-6 -10 -20 -30t-49 -63.5t-74.5 -85.5t-97 -90t-116.5 -83.5t-132.5 -59t-145.5 -23.5 t-145.5 23.5t-132.5 59t-116.5 83.5t-97 90t-74.5 85.5t-49 63.5t-20 30zM120 600q7 -10 40.5 -58t56 -78.5t68 -77.5t87.5 -75t103 -49.5t125 -21.5t123.5 20t100.5 45.5t85.5 71.5t66.5 75.5t58 81.5t47 66q-1 1 -28.5 37.5t-42 55t-43.5 53t-57.5 63.5t-58.5 54 q49 -74 49 -163q0 -124 -88 -212t-212 -88t-212 88t-88 212q0 85 46 158q-102 -87 -226 -258zM377 656q49 -124 154 -191l105 105q-37 24 -75 72t-57 84l-20 36z" />
+<glyph unicode="&#xe106;" d="M-61 600l26 40q6 10 20 30t49 63.5t74.5 85.5t97 90t116.5 83.5t132.5 59t145.5 23.5q61 0 121 -17l37 142h148l-314 -1200h-148l37 143q-82 21 -165 71.5t-140 102t-109.5 112t-72 88.5t-29.5 43zM120 600q210 -282 393 -336l37 141q-107 18 -178.5 101.5t-71.5 193.5 q0 85 46 158q-102 -87 -226 -258zM377 656q49 -124 154 -191l47 47l23 87q-30 28 -59 69t-44 68l-14 26zM780 161l38 145q22 15 44.5 34t46 44t40.5 44t41 50.5t33.5 43.5t33 44t24.5 34q-97 127 -140 175l39 146q67 -54 131.5 -125.5t87.5 -103.5t36 -52l26 -40l-26 -40 q-7 -12 -25.5 -38t-63.5 -79.5t-95.5 -102.5t-124 -100t-146.5 -79z" />
+<glyph unicode="&#xe107;" d="M-97.5 34q13.5 -34 50.5 -34h1294q37 0 50.5 35.5t-7.5 67.5l-642 1056q-20 33 -48 36t-48 -29l-642 -1066q-21 -32 -7.5 -66zM155 200l445 723l445 -723h-345v100h-200v-100h-345zM500 600l100 -300l100 300v100h-200v-100z" />
+<glyph unicode="&#xe108;" d="M100 262v41q0 20 11 44.5t26 38.5l363 325v339q0 62 44 106t106 44t106 -44t44 -106v-339l363 -325q15 -14 26 -38.5t11 -44.5v-41q0 -20 -12 -26.5t-29 5.5l-359 249v-263q100 -91 100 -113v-64q0 -21 -13 -29t-32 1l-94 78h-222l-94 -78q-19 -9 -32 -1t-13 29v64 q0 22 100 113v263l-359 -249q-17 -12 -29 -5.5t-12 26.5z" />
+<glyph unicode="&#xe109;" d="M0 50q0 -20 14.5 -35t35.5 -15h1000q21 0 35.5 15t14.5 35v750h-1100v-750zM0 900h1100v150q0 21 -14.5 35.5t-35.5 14.5h-150v100h-100v-100h-500v100h-100v-100h-150q-21 0 -35.5 -14.5t-14.5 -35.5v-150zM100 100v100h100v-100h-100zM100 300v100h100v-100h-100z M100 500v100h100v-100h-100zM300 100v100h100v-100h-100zM300 300v100h100v-100h-100zM300 500v100h100v-100h-100zM500 100v100h100v-100h-100zM500 300v100h100v-100h-100zM500 500v100h100v-100h-100zM700 100v100h100v-100h-100zM700 300v100h100v-100h-100zM700 500 v100h100v-100h-100zM900 100v100h100v-100h-100zM900 300v100h100v-100h-100zM900 500v100h100v-100h-100z" />
+<glyph unicode="&#xe110;" d="M0 200v200h259l600 600h241v198l300 -295l-300 -300v197h-159l-600 -600h-341zM0 800h259l122 -122l141 142l-181 180h-341v-200zM678 381l141 142l122 -123h159v198l300 -295l-300 -300v197h-241z" />
+<glyph unicode="&#xe111;" d="M0 400v600q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-600q0 -41 -29.5 -70.5t-70.5 -29.5h-596l-304 -300v300h-100q-41 0 -70.5 29.5t-29.5 70.5z" />
+<glyph unicode="&#xe112;" d="M100 600v200h300v-250q0 -113 6 -145q17 -92 102 -117q39 -11 92 -11q37 0 66.5 5.5t50 15.5t36 24t24 31.5t14 37.5t7 42t2.5 45t0 47v25v250h300v-200q0 -42 -3 -83t-15 -104t-31.5 -116t-58 -109.5t-89 -96.5t-129 -65.5t-174.5 -25.5t-174.5 25.5t-129 65.5t-89 96.5 t-58 109.5t-31.5 116t-15 104t-3 83zM100 900v300h300v-300h-300zM800 900v300h300v-300h-300z" />
+<glyph unicode="&#xe113;" d="M-30 411l227 -227l352 353l353 -353l226 227l-578 579z" />
+<glyph unicode="&#xe114;" d="M70 797l580 -579l578 579l-226 227l-353 -353l-352 353z" />
+<glyph unicode="&#xe115;" d="M-198 700l299 283l300 -283h-203v-400h385l215 -200h-800v600h-196zM402 1000l215 -200h381v-400h-198l299 -283l299 283h-200v600h-796z" />
+<glyph unicode="&#xe116;" d="M18 939q-5 24 10 42q14 19 39 19h896l38 162q5 17 18.5 27.5t30.5 10.5h94q20 0 35 -14.5t15 -35.5t-15 -35.5t-35 -14.5h-54l-201 -961q-2 -4 -6 -10.5t-19 -17.5t-33 -11h-31v-50q0 -20 -14.5 -35t-35.5 -15t-35.5 15t-14.5 35v50h-300v-50q0 -20 -14.5 -35t-35.5 -15 t-35.5 15t-14.5 35v50h-50q-21 0 -35.5 15t-14.5 35q0 21 14.5 35.5t35.5 14.5h535l48 200h-633q-32 0 -54.5 21t-27.5 43z" />
+<glyph unicode="&#xe117;" d="M0 0v800h1200v-800h-1200zM0 900v100h200q0 41 29.5 70.5t70.5 29.5h300q41 0 70.5 -29.5t29.5 -70.5h500v-100h-1200z" />
+<glyph unicode="&#xe118;" d="M1 0l300 700h1200l-300 -700h-1200zM1 400v600h200q0 41 29.5 70.5t70.5 29.5h300q41 0 70.5 -29.5t29.5 -70.5h500v-200h-1000z" />
+<glyph unicode="&#xe119;" d="M302 300h198v600h-198l298 300l298 -300h-198v-600h198l-298 -300z" />
+<glyph unicode="&#xe120;" d="M0 600l300 298v-198h600v198l300 -298l-300 -297v197h-600v-197z" />
+<glyph unicode="&#xe121;" d="M0 100v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM31 400l172 739q5 22 23 41.5t38 19.5h672q19 0 37.5 -22.5t23.5 -45.5l172 -732h-1138zM800 100h100v100h-100v-100z M1000 100h100v100h-100v-100z" />
+<glyph unicode="&#xe122;" d="M-101 600v50q0 24 25 49t50 38l25 13v-250l-11 5.5t-24 14t-30 21.5t-24 27.5t-11 31.5zM99 500v250v5q0 13 0.5 18.5t2.5 13t8 10.5t15 3h200l675 250v-850l-675 200h-38l47 -276q2 -12 -3 -17.5t-11 -6t-21 -0.5h-8h-83q-20 0 -34.5 14t-18.5 35q-56 337 -56 351z M1100 200v850q0 21 14.5 35.5t35.5 14.5q20 0 35 -14.5t15 -35.5v-850q0 -20 -15 -35t-35 -15q-21 0 -35.5 15t-14.5 35z" />
+<glyph unicode="&#xe123;" d="M74 350q0 21 13.5 35.5t33.5 14.5h17l118 173l63 327q15 77 76 140t144 83l-18 32q-6 19 3 32t29 13h94q20 0 29 -10.5t3 -29.5l-18 -37q83 -19 144 -82.5t76 -140.5l63 -327l118 -173h17q20 0 33.5 -14.5t13.5 -35.5q0 -20 -13 -40t-31 -27q-22 -9 -63 -23t-167.5 -37 t-251.5 -23t-245.5 20.5t-178.5 41.5l-58 20q-18 7 -31 27.5t-13 40.5zM497 110q12 -49 40 -79.5t63 -30.5t63 30.5t39 79.5q-48 -6 -102 -6t-103 6z" />
+<glyph unicode="&#xe124;" d="M21 445l233 -45l-78 -224l224 78l45 -233l155 179l155 -179l45 233l224 -78l-78 224l234 45l-180 155l180 156l-234 44l78 225l-224 -78l-45 233l-155 -180l-155 180l-45 -233l-224 78l78 -225l-233 -44l179 -156z" />
+<glyph unicode="&#xe125;" d="M0 200h200v600h-200v-600zM300 275q0 -75 100 -75h61q123 -100 139 -100h250q46 0 83 57l238 344q29 31 29 74v100q0 44 -30.5 84.5t-69.5 40.5h-328q28 118 28 125v150q0 44 -30.5 84.5t-69.5 40.5h-50q-27 0 -51 -20t-38 -48l-96 -198l-145 -196q-20 -26 -20 -63v-400z M400 300v375l150 212l100 213h50v-175l-50 -225h450v-125l-250 -375h-214l-136 100h-100z" />
+<glyph unicode="&#xe126;" d="M0 400v600h200v-600h-200zM300 525v400q0 75 100 75h61q123 100 139 100h250q46 0 83 -57l238 -344q29 -31 29 -74v-100q0 -44 -30.5 -84.5t-69.5 -40.5h-328q28 -118 28 -125v-150q0 -44 -30.5 -84.5t-69.5 -40.5h-50q-27 0 -51 20t-38 48l-96 198l-145 196 q-20 26 -20 63zM400 525l150 -212l100 -213h50v175l-50 225h450v125l-250 375h-214l-136 -100h-100v-375z" />
+<glyph unicode="&#xe127;" d="M8 200v600h200v-600h-200zM308 275v525q0 17 14 35.5t28 28.5l14 9l362 230q14 6 25 6q17 0 29 -12l109 -112q14 -14 14 -34q0 -18 -11 -32l-85 -121h302q85 0 138.5 -38t53.5 -110t-54.5 -111t-138.5 -39h-107l-130 -339q-7 -22 -20.5 -41.5t-28.5 -19.5h-341 q-7 0 -90 81t-83 94zM408 289l100 -89h293l131 339q6 21 19.5 41t28.5 20h203q16 0 25 15t9 36q0 20 -9 34.5t-25 14.5h-457h-6.5h-7.5t-6.5 0.5t-6 1t-5 1.5t-5.5 2.5t-4 4t-4 5.5q-5 12 -5 20q0 14 10 27l147 183l-86 83l-339 -236v-503z" />
+<glyph unicode="&#xe128;" d="M-101 651q0 72 54 110t139 37h302l-85 121q-11 16 -11 32q0 21 14 34l109 113q13 12 29 12q11 0 25 -6l365 -230q7 -4 16.5 -10.5t26 -26t16.5 -36.5v-526q0 -13 -85.5 -93.5t-93.5 -80.5h-342q-15 0 -28.5 20t-19.5 41l-131 339h-106q-84 0 -139 39t-55 111zM-1 601h222 q15 0 28.5 -20.5t19.5 -40.5l131 -339h293l106 89v502l-342 237l-87 -83l145 -184q10 -11 10 -26q0 -11 -5 -20q-1 -3 -3.5 -5.5l-4 -4t-5 -2.5t-5.5 -1.5t-6.5 -1t-6.5 -0.5h-7.5h-6.5h-476v-100zM999 201v600h200v-600h-200z" />
+<glyph unicode="&#xe129;" d="M97 719l230 -363q4 -6 10.5 -15.5t26 -25t36.5 -15.5h525q13 0 94 83t81 90v342q0 15 -20 28.5t-41 19.5l-339 131v106q0 84 -39 139t-111 55t-110 -53.5t-38 -138.5v-302l-121 84q-15 12 -33.5 11.5t-32.5 -13.5l-112 -110q-22 -22 -6 -53zM172 739l83 86l183 -146 q22 -18 47 -5q3 1 5.5 3.5l4 4t2.5 5t1.5 5.5t1 6.5t0.5 6v7.5v7v456q0 22 25 31t50 -0.5t25 -30.5v-202q0 -16 20 -29.5t41 -19.5l339 -130v-294l-89 -100h-503zM400 0v200h600v-200h-600z" />
+<glyph unicode="&#xe130;" d="M1 585q-15 -31 7 -53l112 -110q13 -13 32 -13.5t34 10.5l121 85l-1 -302q0 -84 38.5 -138t110.5 -54t111 55t39 139v106l339 131q20 6 40.5 19.5t20.5 28.5v342q0 7 -81 90t-94 83h-525q-17 0 -35.5 -14t-28.5 -28l-10 -15zM76 565l237 339h503l89 -100v-294l-340 -130 q-20 -6 -40 -20t-20 -29v-202q0 -22 -25 -31t-50 0t-25 31v456v14.5t-1.5 11.5t-5 12t-9.5 7q-24 13 -46 -5l-184 -146zM305 1104v200h600v-200h-600z" />
+<glyph unicode="&#xe131;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q162 0 299.5 -80t217.5 -218t80 -300t-80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM300 500h300l-2 -194l402 294l-402 298v-197h-298v-201z" />
+<glyph unicode="&#xe132;" d="M0 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t231.5 47.5q122 0 232.5 -47.5t190.5 -127.5t127.5 -190.5t47.5 -232.5q0 -162 -80 -299.5t-218 -217.5t-300 -80t-299.5 80t-217.5 217.5t-80 299.5zM200 600l400 -294v194h302v201h-300v197z" />
+<glyph unicode="&#xe133;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q121 0 231.5 -47.5t190.5 -127.5t127.5 -190.5t47.5 -232.5q0 -162 -80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM300 600h200v-300h200v300h200l-300 400z" />
+<glyph unicode="&#xe134;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q121 0 231.5 -47.5t190.5 -127.5t127.5 -190.5t47.5 -232.5q0 -162 -80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM300 600l300 -400l300 400h-200v300h-200v-300h-200z" />
+<glyph unicode="&#xe135;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q121 0 231.5 -47.5t190.5 -127.5t127.5 -190.5t47.5 -232.5q0 -162 -80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM254 780q-8 -34 5.5 -93t7.5 -87q0 -9 17 -44t16 -60q12 0 23 -5.5 t23 -15t20 -13.5q20 -10 108 -42q22 -8 53 -31.5t59.5 -38.5t57.5 -11q8 -18 -15 -55.5t-20 -57.5q12 -21 22.5 -34.5t28 -27t36.5 -17.5q0 -6 -3 -15.5t-3.5 -14.5t4.5 -17q101 -2 221 111q31 30 47 48t34 49t21 62q-14 9 -37.5 9.5t-35.5 7.5q-14 7 -49 15t-52 19 q-9 0 -39.5 -0.5t-46.5 -1.5t-39 -6.5t-39 -16.5q-50 -35 -66 -12q-4 2 -3.5 25.5t0.5 25.5q-6 13 -26.5 17t-24.5 7q2 22 -2 41t-16.5 28t-38.5 -20q-23 -25 -42 4q-19 28 -8 58q8 16 22 22q6 -1 26 -1.5t33.5 -4.5t19.5 -13q12 -19 32 -37.5t34 -27.5l14 -8q0 3 9.5 39.5 t5.5 57.5q-4 23 14.5 44.5t22.5 31.5q5 14 10 35t8.5 31t15.5 22.5t34 21.5q-6 18 10 37q8 0 23.5 -1.5t24.5 -1.5t20.5 4.5t20.5 15.5q-10 23 -30.5 42.5t-38 30t-49 26.5t-43.5 23q11 41 1 44q31 -13 58.5 -14.5t39.5 3.5l11 4q6 36 -17 53.5t-64 28.5t-56 23 q-19 -3 -37 0q-15 -12 -36.5 -21t-34.5 -12t-44 -8t-39 -6q-15 -3 -46 0t-45 -3q-20 -6 -51.5 -25.5t-34.5 -34.5q-3 -11 6.5 -22.5t8.5 -18.5q-3 -34 -27.5 -91t-29.5 -79zM518 915q3 12 16 30.5t16 25.5q10 -10 18.5 -10t14 6t14.5 14.5t16 12.5q0 -18 8 -42.5t16.5 -44 t9.5 -23.5q-6 1 -39 5t-53.5 10t-36.5 16z" />
+<glyph unicode="&#xe136;" d="M0 164.5q0 21.5 15 37.5l600 599q-33 101 6 201.5t135 154.5q164 92 306 -9l-259 -138l145 -232l251 126q13 -175 -151 -267q-123 -70 -253 -23l-596 -596q-15 -16 -36.5 -16t-36.5 16l-111 110q-15 15 -15 36.5z" />
+<glyph unicode="&#xe137;" horiz-adv-x="1220" d="M0 196v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM0 596v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000 q-41 0 -70.5 29.5t-29.5 70.5zM0 996v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM600 596h500v100h-500v-100zM800 196h300v100h-300v-100zM900 996h200v100h-200v-100z" />
+<glyph unicode="&#xe138;" d="M100 1100v100h1000v-100h-1000zM150 1000h900l-350 -500v-300l-200 -200v500z" />
+<glyph unicode="&#xe139;" d="M0 200v200h1200v-200q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM0 500v400q0 41 29.5 70.5t70.5 29.5h300v100q0 41 29.5 70.5t70.5 29.5h200q41 0 70.5 -29.5t29.5 -70.5v-100h300q41 0 70.5 -29.5t29.5 -70.5v-400h-500v100h-200v-100h-500z M500 1000h200v100h-200v-100z" />
+<glyph unicode="&#xe140;" d="M0 0v400l129 -129l200 200l142 -142l-200 -200l129 -129h-400zM0 800l129 129l200 -200l142 142l-200 200l129 129h-400v-400zM729 329l142 142l200 -200l129 129v-400h-400l129 129zM729 871l200 200l-129 129h400v-400l-129 129l-200 -200z" />
+<glyph unicode="&#xe141;" d="M0 596q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM182 596q0 -172 121.5 -293t292.5 -121t292.5 121t121.5 293q0 171 -121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM291 655 q0 23 15.5 38.5t38.5 15.5t39 -16t16 -38q0 -23 -16 -39t-39 -16q-22 0 -38 16t-16 39zM400 850q0 22 16 38.5t39 16.5q22 0 38 -16t16 -39t-16 -39t-38 -16q-23 0 -39 16.5t-16 38.5zM513 609q0 32 21 56.5t52 29.5l122 126l1 1q-9 14 -9 28q0 22 16 38.5t39 16.5 q22 0 38 -16t16 -39t-16 -39t-38 -16q-16 0 -29 10l-55 -145q17 -22 17 -51q0 -36 -25.5 -61.5t-61.5 -25.5q-37 0 -62.5 25.5t-25.5 61.5zM800 655q0 22 16 38t39 16t38.5 -15.5t15.5 -38.5t-16 -39t-38 -16q-23 0 -39 16t-16 39z" />
+<glyph unicode="&#xe142;" d="M-40 375q-13 -95 35 -173q35 -57 94 -89t129 -32q63 0 119 28q33 16 65 40.5t52.5 45.5t59.5 64q40 44 57 61l394 394q35 35 47 84t-3 96q-27 87 -117 104q-20 2 -29 2q-46 0 -79.5 -17t-67.5 -51l-388 -396l-7 -7l69 -67l377 373q20 22 39 38q23 23 50 23q38 0 53 -36 q16 -39 -20 -75l-547 -547q-52 -52 -125 -52q-55 0 -100 33t-54 96q-5 35 2.5 66t31.5 63t42 50t56 54q24 21 44 41l348 348q52 52 82.5 79.5t84 54t107.5 26.5q25 0 48 -4q95 -17 154 -94.5t51 -175.5q-7 -101 -98 -192l-252 -249l-253 -256l7 -7l69 -60l517 511 q67 67 95 157t11 183q-16 87 -67 154t-130 103q-69 33 -152 33q-107 0 -197 -55q-40 -24 -111 -95l-512 -512q-68 -68 -81 -163z" />
+<glyph unicode="&#xe143;" d="M79 784q0 131 99 229.5t230 98.5q144 0 242 -129q103 129 245 129q130 0 227 -98.5t97 -229.5q0 -46 -17.5 -91t-61 -99t-77 -89.5t-104.5 -105.5q-197 -191 -293 -322l-17 -23l-16 23q-43 58 -100 122.5t-92 99.5t-101 100l-84.5 84.5t-68 74t-60 78t-33.5 70.5t-15 78z M250 784q0 -27 30.5 -70t61.5 -75.5t95 -94.5l22 -22q93 -90 190 -201q82 92 195 203l12 12q64 62 97.5 97t64.5 79t31 72q0 71 -48 119.5t-106 48.5q-73 0 -131 -83l-118 -171l-114 174q-51 80 -124 80q-59 0 -108.5 -49.5t-49.5 -118.5z" />
+<glyph unicode="&#xe144;" d="M57 353q0 -94 66 -160l141 -141q66 -66 159 -66q95 0 159 66l283 283q66 66 66 159t-66 159l-141 141q-12 12 -19 17l-105 -105l212 -212l-389 -389l-247 248l95 95l-18 18q-46 45 -75 101l-55 -55q-66 -66 -66 -159zM269 706q0 -93 66 -159l141 -141l19 -17l105 105 l-212 212l389 389l247 -247l-95 -96l18 -18q46 -46 77 -99l29 29q35 35 62.5 88t27.5 96q0 93 -66 159l-141 141q-66 66 -159 66q-95 0 -159 -66l-283 -283q-66 -64 -66 -159z" />
+<glyph unicode="&#xe145;" d="M200 100v953q0 21 30 46t81 48t129 38t163 15t162 -15t127 -38t79 -48t29 -46v-953q0 -41 -29.5 -70.5t-70.5 -29.5h-600q-41 0 -70.5 29.5t-29.5 70.5zM300 300h600v700h-600v-700zM496 150q0 -43 30.5 -73.5t73.5 -30.5t73.5 30.5t30.5 73.5t-30.5 73.5t-73.5 30.5 t-73.5 -30.5t-30.5 -73.5z" />
+<glyph unicode="&#xe146;" d="M0 0l303 380l207 208l-210 212h300l267 279l-35 36q-15 14 -15 35t15 35q14 15 35 15t35 -15l283 -282q15 -15 15 -36t-15 -35q-14 -15 -35 -15t-35 15l-36 35l-279 -267v-300l-212 210l-208 -207z" />
+<glyph unicode="&#xe148;" d="M295 433h139q5 -77 48.5 -126.5t117.5 -64.5v335l-27 7q-46 14 -79 26.5t-72 36t-62.5 52t-40 72.5t-16.5 99q0 92 44 159.5t109 101t144 40.5v78h100v-79q38 -4 72.5 -13.5t75.5 -31.5t71 -53.5t51.5 -84t24.5 -118.5h-159q-8 72 -35 109.5t-101 50.5v-307l64 -14 q34 -7 64 -16.5t70 -31.5t67.5 -52t47.5 -80.5t20 -112.5q0 -139 -89 -224t-244 -96v-77h-100v78q-152 17 -237 104q-40 40 -52.5 93.5t-15.5 139.5zM466 889q0 -29 8 -51t16.5 -34t29.5 -22.5t31 -13.5t38 -10q7 -2 11 -3v274q-61 -8 -97.5 -37.5t-36.5 -102.5zM700 237 q170 18 170 151q0 64 -44 99.5t-126 60.5v-311z" />
+<glyph unicode="&#xe149;" d="M100 600v100h166q-24 49 -44 104q-10 26 -14.5 55.5t-3 72.5t25 90t68.5 87q97 88 263 88q129 0 230 -89t101 -208h-153q0 52 -34 89.5t-74 51.5t-76 14q-37 0 -79 -14.5t-62 -35.5q-41 -44 -41 -101q0 -11 2.5 -24.5t5.5 -24t9.5 -26.5t10.5 -25t14 -27.5t14 -25.5 t15.5 -27t13.5 -24h242v-100h-197q8 -50 -2.5 -115t-31.5 -94q-41 -59 -99 -113q35 11 84 18t70 7q32 1 102 -16t104 -17q76 0 136 30l50 -147q-41 -25 -80.5 -36.5t-59 -13t-61.5 -1.5q-23 0 -128 33t-155 29q-39 -4 -82 -17t-66 -25l-24 -11l-55 145l16.5 11t15.5 10 t13.5 9.5t14.5 12t14.5 14t17.5 18.5q48 55 54 126.5t-30 142.5h-221z" />
+<glyph unicode="&#xe150;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM602 900l298 300l298 -300h-198v-900h-200v900h-198z" />
+<glyph unicode="&#xe151;" d="M2 300h198v900h200v-900h198l-298 -300zM700 0v200h100v-100h200v-100h-300zM700 400v100h300v-200h-99v-100h-100v100h99v100h-200zM700 700v500h300v-500h-100v100h-100v-100h-100zM801 900h100v200h-100v-200z" />
+<glyph unicode="&#xe152;" d="M2 300h198v900h200v-900h198l-298 -300zM700 0v500h300v-500h-100v100h-100v-100h-100zM700 700v200h100v-100h200v-100h-300zM700 1100v100h300v-200h-99v-100h-100v100h99v100h-200zM801 200h100v200h-100v-200z" />
+<glyph unicode="&#xe153;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM800 100v400h300v-500h-100v100h-200zM800 1100v100h200v-500h-100v400h-100zM901 200h100v200h-100v-200z" />
+<glyph unicode="&#xe154;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM800 400v100h200v-500h-100v400h-100zM800 800v400h300v-500h-100v100h-200zM901 900h100v200h-100v-200z" />
+<glyph unicode="&#xe155;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM700 100v200h500v-200h-500zM700 400v200h400v-200h-400zM700 700v200h300v-200h-300zM700 1000v200h200v-200h-200z" />
+<glyph unicode="&#xe156;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM700 100v200h200v-200h-200zM700 400v200h300v-200h-300zM700 700v200h400v-200h-400zM700 1000v200h500v-200h-500z" />
+<glyph unicode="&#xe157;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h300q162 0 281 -118.5t119 -281.5v-300q0 -165 -118.5 -282.5t-281.5 -117.5h-300q-165 0 -282.5 117.5t-117.5 282.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500z" />
+<glyph unicode="&#xe158;" d="M0 400v300q0 163 119 281.5t281 118.5h300q165 0 282.5 -117.5t117.5 -282.5v-300q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-163 0 -281.5 117.5t-118.5 282.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM400 300l333 250l-333 250v-500z" />
+<glyph unicode="&#xe159;" d="M0 400v300q0 163 117.5 281.5t282.5 118.5h300q163 0 281.5 -119t118.5 -281v-300q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5t-117.5 282.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM300 700l250 -333l250 333h-500z" />
+<glyph unicode="&#xe160;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h300q165 0 282.5 -117.5t117.5 -282.5v-300q0 -162 -118.5 -281t-281.5 -119h-300q-165 0 -282.5 118.5t-117.5 281.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM300 400h500l-250 333z" />
+<glyph unicode="&#xe161;" d="M0 400v300h300v200l400 -350l-400 -350v200h-300zM500 0v200h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5h-500v200h400q165 0 282.5 -117.5t117.5 -282.5v-300q0 -165 -117.5 -282.5t-282.5 -117.5h-400z" />
+<glyph unicode="&#xe162;" d="M216 519q10 -19 32 -19h302q-155 -438 -160 -458q-5 -21 4 -32l9 -8l9 -1q13 0 26 16l538 630q15 19 6 36q-8 18 -32 16h-300q1 4 78 219.5t79 227.5q2 17 -6 27l-8 8h-9q-16 0 -25 -15q-4 -5 -98.5 -111.5t-228 -257t-209.5 -238.5q-17 -19 -7 -40z" />
+<glyph unicode="&#xe163;" d="M0 400q0 -165 117.5 -282.5t282.5 -117.5h300q47 0 100 15v185h-500q-41 0 -70.5 29.5t-29.5 70.5v500q0 41 29.5 70.5t70.5 29.5h500v185q-14 4 -114 7.5t-193 5.5l-93 2q-165 0 -282.5 -117.5t-117.5 -282.5v-300zM600 400v300h300v200l400 -350l-400 -350v200h-300z " />
+<glyph unicode="&#xe164;" d="M0 400q0 -165 117.5 -282.5t282.5 -117.5h300q163 0 281.5 117.5t118.5 282.5v98l-78 73l-122 -123v-148q0 -41 -29.5 -70.5t-70.5 -29.5h-500q-41 0 -70.5 29.5t-29.5 70.5v500q0 41 29.5 70.5t70.5 29.5h156l118 122l-74 78h-100q-165 0 -282.5 -117.5t-117.5 -282.5 v-300zM496 709l353 342l-149 149h500v-500l-149 149l-342 -353z" />
+<glyph unicode="&#xe165;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM406 600 q0 80 57 137t137 57t137 -57t57 -137t-57 -137t-137 -57t-137 57t-57 137z" />
+<glyph unicode="&#xe166;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 800l445 -500l450 500h-295v400h-300v-400h-300zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe167;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 700h300v-300h300v300h295l-445 500zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe168;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 705l305 -305l596 596l-154 155l-442 -442l-150 151zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe169;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 988l97 -98l212 213l-97 97zM200 401h700v699l-250 -239l-149 149l-212 -212l149 -149zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe170;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM200 612l212 -212l98 97l-213 212zM300 1200l239 -250l-149 -149l212 -212l149 148l248 -237v700h-699zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe171;" d="M23 415l1177 784v-1079l-475 272l-310 -393v416h-392zM494 210l672 938l-672 -712v-226z" />
+<glyph unicode="&#xe172;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-850q0 -21 -15 -35.5t-35 -14.5h-150v400h-700v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 1000h100v200h-100v-200z" />
+<glyph unicode="&#xe173;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-218l-276 -275l-120 120l-126 -127h-378v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM581 306l123 123l120 -120l353 352l123 -123l-475 -476zM600 1000h100v200h-100v-200z" />
+<glyph unicode="&#xe174;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-269l-103 -103l-170 170l-298 -298h-329v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 1000h100v200h-100v-200zM700 133l170 170l-170 170l127 127l170 -170l170 170l127 -128l-170 -169l170 -170 l-127 -127l-170 170l-170 -170z" />
+<glyph unicode="&#xe175;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-300h-400v-200h-500v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 300l300 -300l300 300h-200v300h-200v-300h-200zM600 1000v200h100v-200h-100z" />
+<glyph unicode="&#xe176;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-402l-200 200l-298 -298h-402v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 300h200v-300h200v300h200l-300 300zM600 1000v200h100v-200h-100z" />
+<glyph unicode="&#xe177;" d="M0 250q0 -21 14.5 -35.5t35.5 -14.5h1100q21 0 35.5 14.5t14.5 35.5v550h-1200v-550zM0 900h1200v150q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5v-150zM100 300v200h400v-200h-400z" />
+<glyph unicode="&#xe178;" d="M0 400l300 298v-198h400v-200h-400v-198zM100 800v200h100v-200h-100zM300 800v200h100v-200h-100zM500 800v200h400v198l300 -298l-300 -298v198h-400zM800 300v200h100v-200h-100zM1000 300h100v200h-100v-200z" />
+<glyph unicode="&#xe179;" d="M100 700v400l50 100l50 -100v-300h100v300l50 100l50 -100v-300h100v300l50 100l50 -100v-400l-100 -203v-447q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5v447zM800 597q0 -29 10.5 -55.5t25 -43t29 -28.5t25.5 -18l10 -5v-397q0 -21 14.5 -35.5 t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v1106q0 31 -18 40.5t-44 -7.5l-276 -117q-25 -16 -43.5 -50.5t-18.5 -65.5v-359z" />
+<glyph unicode="&#xe180;" d="M100 0h400v56q-75 0 -87.5 6t-12.5 44v394h500v-394q0 -38 -12.5 -44t-87.5 -6v-56h400v56q-4 0 -11 0.5t-24 3t-30 7t-24 15t-11 24.5v888q0 22 25 34.5t50 13.5l25 2v56h-400v-56q75 0 87.5 -6t12.5 -44v-394h-500v394q0 38 12.5 44t87.5 6v56h-400v-56q4 0 11 -0.5 t24 -3t30 -7t24 -15t11 -24.5v-888q0 -22 -25 -34.5t-50 -13.5l-25 -2v-56z" />
+<glyph unicode="&#xe181;" d="M0 300q0 -41 29.5 -70.5t70.5 -29.5h300q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5h-300q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM100 100h400l200 200h105l295 98v-298h-425l-100 -100h-375zM100 300v200h300v-200h-300zM100 600v200h300v-200h-300z M100 1000h400l200 -200v-98l295 98h105v200h-425l-100 100h-375zM700 402v163l400 133v-163z" />
+<glyph unicode="&#xe182;" d="M16.5 974.5q0.5 -21.5 16 -90t46.5 -140t104 -177.5t175 -208q103 -103 207.5 -176t180 -103.5t137 -47t92.5 -16.5l31 1l163 162q16 17 13 40.5t-22 37.5l-192 136q-19 14 -45 12t-42 -19l-119 -118q-143 103 -267 227q-126 126 -227 268l118 118q17 17 20 41.5 t-11 44.5l-139 194q-14 19 -36.5 22t-40.5 -14l-162 -162q-1 -11 -0.5 -32.5z" />
+<glyph unicode="&#xe183;" d="M0 50v212q0 20 10.5 45.5t24.5 39.5l365 303v50q0 4 1 10.5t12 22.5t30 28.5t60 23t97 10.5t97 -10t60 -23.5t30 -27.5t12 -24l1 -10v-50l365 -303q14 -14 24.5 -39.5t10.5 -45.5v-212q0 -21 -15 -35.5t-35 -14.5h-1100q-21 0 -35.5 14.5t-14.5 35.5zM0 712 q0 -21 14.5 -33.5t34.5 -8.5l202 33q20 4 34.5 21t14.5 38v146q141 24 300 24t300 -24v-146q0 -21 14.5 -38t34.5 -21l202 -33q20 -4 34.5 8.5t14.5 33.5v200q-6 8 -19 20.5t-63 45t-112 57t-171 45t-235 20.5q-92 0 -175 -10.5t-141.5 -27t-108.5 -36.5t-81.5 -40 t-53.5 -36.5t-31 -27.5l-9 -10v-200z" />
+<glyph unicode="&#xe184;" d="M100 0v100h1100v-100h-1100zM175 200h950l-125 150v250l100 100v400h-100v-200h-100v200h-200v-200h-100v200h-200v-200h-100v200h-100v-400l100 -100v-250z" />
+<glyph unicode="&#xe185;" d="M100 0h300v400q0 41 -29.5 70.5t-70.5 29.5h-100q-41 0 -70.5 -29.5t-29.5 -70.5v-400zM500 0v1000q0 41 29.5 70.5t70.5 29.5h100q41 0 70.5 -29.5t29.5 -70.5v-1000h-300zM900 0v700q0 41 29.5 70.5t70.5 29.5h100q41 0 70.5 -29.5t29.5 -70.5v-700h-300z" />
+<glyph unicode="&#xe186;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v300h-200v100h200v100h-300v-300h200v-100h-200v-100zM600 300h200v100h100v300h-100v100h-200v-500 zM700 400v300h100v-300h-100z" />
+<glyph unicode="&#xe187;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h100v200h100v-200h100v500h-100v-200h-100v200h-100v-500zM600 300h200v100h100v300h-100v100h-200v-500 zM700 400v300h100v-300h-100z" />
+<glyph unicode="&#xe188;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v100h-200v300h200v100h-300v-500zM600 300h300v100h-200v300h200v100h-300v-500z" />
+<glyph unicode="&#xe189;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 550l300 -150v300zM600 400l300 150l-300 150v-300z" />
+<glyph unicode="&#xe190;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300v500h700v-500h-700zM300 400h130q41 0 68 42t27 107t-28.5 108t-66.5 43h-130v-300zM575 549 q0 -65 27 -107t68 -42h130v300h-130q-38 0 -66.5 -43t-28.5 -108z" />
+<glyph unicode="&#xe191;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v300h-200v100h200v100h-300v-300h200v-100h-200v-100zM601 300h100v100h-100v-100zM700 700h100 v-400h100v500h-200v-100z" />
+<glyph unicode="&#xe192;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v400h-200v100h-100v-500zM301 400v200h100v-200h-100zM601 300h100v100h-100v-100zM700 700h100 v-400h100v500h-200v-100z" />
+<glyph unicode="&#xe193;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 700v100h300v-300h-99v-100h-100v100h99v200h-200zM201 300v100h100v-100h-100zM601 300v100h100v-100h-100z M700 700v100h200v-500h-100v400h-100z" />
+<glyph unicode="&#xe194;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM400 500v200 l100 100h300v-100h-300v-200h300v-100h-300z" />
+<glyph unicode="&#xe195;" d="M0 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM182 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM400 400v400h300 l100 -100v-100h-100v100h-200v-100h200v-100h-200v-100h-100zM700 400v100h100v-100h-100z" />
+<glyph unicode="&#xe197;" d="M-14 494q0 -80 56.5 -137t135.5 -57h222v300h400v-300h128q120 0 205 86t85 208q0 120 -85 206.5t-205 86.5q-46 0 -90 -14q-44 97 -134.5 156.5t-200.5 59.5q-152 0 -260 -107.5t-108 -260.5q0 -25 2 -37q-66 -14 -108.5 -67.5t-42.5 -122.5zM300 200h200v300h200v-300 h200l-300 -300z" />
+<glyph unicode="&#xe198;" d="M-14 494q0 -80 56.5 -137t135.5 -57h8l414 414l403 -403q94 26 154.5 104t60.5 178q0 121 -85 207.5t-205 86.5q-46 0 -90 -14q-44 97 -134.5 156.5t-200.5 59.5q-152 0 -260 -107.5t-108 -260.5q0 -25 2 -37q-66 -14 -108.5 -67.5t-42.5 -122.5zM300 200l300 300 l300 -300h-200v-300h-200v300h-200z" />
+<glyph unicode="&#xe199;" d="M100 200h400v-155l-75 -45h350l-75 45v155h400l-270 300h170l-270 300h170l-300 333l-300 -333h170l-270 -300h170z" />
+<glyph unicode="&#xe200;" d="M121 700q0 -53 28.5 -97t75.5 -65q-4 -16 -4 -38q0 -74 52.5 -126.5t126.5 -52.5q56 0 100 30v-306l-75 -45h350l-75 45v306q46 -30 100 -30q74 0 126.5 52.5t52.5 126.5q0 24 -9 55q50 32 79.5 83t29.5 112q0 90 -61.5 155.5t-150.5 71.5q-26 89 -99.5 145.5 t-167.5 56.5q-116 0 -197.5 -81.5t-81.5 -197.5q0 -4 1 -12t1 -11q-14 2 -23 2q-74 0 -126.5 -52.5t-52.5 -126.5z" />
+</font>
+</defs></svg> 

BIN
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.ttf


BIN
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/fonts/glyphicons-halflings-regular.woff


File diff suppressed because it is too large
+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2/js/bootstrap.min.js


File diff suppressed because it is too large
+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js


+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.PageRounder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.PageRounder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
@@ -91,6 +92,10 @@ public class TestFsDatasetCache {
   private static PageRounder rounder = new PageRounder();
   private static PageRounder rounder = new PageRounder();
   private static CacheManipulator prevCacheManipulator;
   private static CacheManipulator prevCacheManipulator;
 
 
+  static {
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(false);
+  }
+
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     assumeTrue(!Path.WINDOWS);
     assumeTrue(!Path.WINDOWS);

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java

@@ -442,6 +442,7 @@ public class TestListCorruptFileBlocks {
     
     
   /**
   /**
    * Test if NN.listCorruptFiles() returns the right number of results.
    * Test if NN.listCorruptFiles() returns the right number of results.
+   * The corrupt blocks are detected by the BlockPoolSliceScanner.
    * Also, test that DFS.listCorruptFileBlocks can make multiple successive
    * Also, test that DFS.listCorruptFileBlocks can make multiple successive
    * calls.
    * calls.
    */
    */
@@ -450,7 +451,6 @@ public class TestListCorruptFileBlocks {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
     try {
     try {
       Configuration conf = new HdfsConfiguration();
       Configuration conf = new HdfsConfiguration();
-      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 15); // datanode scans directories
       conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000); // datanode sends block reports
       conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000); // datanode sends block reports
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster = new MiniDFSCluster.Builder(conf).build();
       FileSystem fs = cluster.getFileSystem();
       FileSystem fs = cluster.getFileSystem();
@@ -491,6 +491,13 @@ public class TestListCorruptFileBlocks {
         }
         }
       }
       }
 
 
+      // Occasionally the BlockPoolSliceScanner can run before we have removed
+      // the blocks. Restart the Datanode to trigger the scanner into running
+      // once more.
+      LOG.info("Restarting Datanode to trigger BlockPoolSliceScanner");
+      cluster.restartDataNodes();
+      cluster.waitActive();
+
       badFiles = 
       badFiles = 
         namenode.getNamesystem().listCorruptFileBlocks("/srcdat2", null);
         namenode.getNamesystem().listCorruptFileBlocks("/srcdat2", null);
         
         

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
 import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -85,6 +86,10 @@ public class TestPathBasedCacheRequests {
   static private NamenodeProtocols proto;
   static private NamenodeProtocols proto;
   static private CacheManipulator prevCacheManipulator;
   static private CacheManipulator prevCacheManipulator;
 
 
+  static {
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(false);
+  }
+
   @Before
   @Before
   public void setup() throws Exception {
   public void setup() throws Exception {
     conf = new HdfsConfiguration();
     conf = new HdfsConfiguration();
@@ -510,8 +515,9 @@ public class TestPathBasedCacheRequests {
     // Create some cache entries
     // Create some cache entries
     int numEntries = 10;
     int numEntries = 10;
     String entryPrefix = "/party-";
     String entryPrefix = "/party-";
+    long prevId = -1;
     for (int i=0; i<numEntries; i++) {
     for (int i=0; i<numEntries; i++) {
-      dfs.addPathBasedCacheDirective(
+      prevId = dfs.addPathBasedCacheDirective(
           new PathBasedCacheDirective.Builder().
           new PathBasedCacheDirective.Builder().
             setPath(new Path(entryPrefix + i)).setPool(pool).build());
             setPath(new Path(entryPrefix + i)).setPool(pool).build());
     }
     }
@@ -549,6 +555,11 @@ public class TestPathBasedCacheRequests {
       assertEquals(pool, cd.getPool());
       assertEquals(pool, cd.getPool());
     }
     }
     assertFalse("Unexpected # of cache directives found", dit.hasNext());
     assertFalse("Unexpected # of cache directives found", dit.hasNext());
+
+    long nextId = dfs.addPathBasedCacheDirective(
+          new PathBasedCacheDirective.Builder().
+            setPath(new Path("/foobar")).setPool(pool).build());
+    assertEquals(prevId + 1, nextId);
   }
   }
 
 
   /**
   /**

+ 97 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java

@@ -28,6 +28,11 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.security.AccessControlException;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -76,6 +81,47 @@ public class TestOpenFilesWithSnapshot {
     cluster.restartNameNode();
     cluster.restartNameNode();
   }
   }
 
 
+  @Test
+  public void testWithCheckpoint() throws Exception {
+    Path path = new Path("/test");
+    doWriteAndAbort(fs, path);
+    fs.delete(new Path("/test/test"), true);
+    NameNode nameNode = cluster.getNameNode();
+    NameNodeAdapter.enterSafeMode(nameNode, false);
+    NameNodeAdapter.saveNamespace(nameNode);
+    NameNodeAdapter.leaveSafeMode(nameNode);
+    cluster.restartNameNode(true);
+    
+    // read snapshot file after restart
+    String test2snapshotPath = Snapshot.getSnapshotPath(path.toString(),
+        "s1/test/test2");
+    DFSTestUtil.readFile(fs, new Path(test2snapshotPath));
+    String test3snapshotPath = Snapshot.getSnapshotPath(path.toString(),
+        "s1/test/test3");
+    DFSTestUtil.readFile(fs, new Path(test3snapshotPath));
+  }
+
+  @Test
+  public void testFilesDeletionWithCheckpoint() throws Exception {
+    Path path = new Path("/test");
+    doWriteAndAbort(fs, path);
+    fs.delete(new Path("/test/test/test2"), true);
+    fs.delete(new Path("/test/test/test3"), true);
+    NameNode nameNode = cluster.getNameNode();
+    NameNodeAdapter.enterSafeMode(nameNode, false);
+    NameNodeAdapter.saveNamespace(nameNode);
+    NameNodeAdapter.leaveSafeMode(nameNode);
+    cluster.restartNameNode(true);
+    
+    // read snapshot file after restart
+    String test2snapshotPath = Snapshot.getSnapshotPath(path.toString(),
+        "s1/test/test2");
+    DFSTestUtil.readFile(fs, new Path(test2snapshotPath));
+    String test3snapshotPath = Snapshot.getSnapshotPath(path.toString(),
+        "s1/test/test3");
+    DFSTestUtil.readFile(fs, new Path(test3snapshotPath));
+  }
+
   private void doWriteAndAbort(DistributedFileSystem fs, Path path)
   private void doWriteAndAbort(DistributedFileSystem fs, Path path)
       throws IOException {
       throws IOException {
     fs.mkdirs(path);
     fs.mkdirs(path);
@@ -110,4 +156,55 @@ public class TestOpenFilesWithSnapshot {
     DFSTestUtil.abortStream((DFSOutputStream) out2.getWrappedStream());
     DFSTestUtil.abortStream((DFSOutputStream) out2.getWrappedStream());
     fs.createSnapshot(path, "s1");
     fs.createSnapshot(path, "s1");
   }
   }
+
+  @Test
+  public void testOpenFilesWithMultipleSnapshots() throws Exception {
+    doTestMultipleSnapshots(true);
+  }
+
+  @Test
+  public void testOpenFilesWithMultipleSnapshotsWithoutCheckpoint()
+      throws Exception {
+    doTestMultipleSnapshots(false);
+  }
+
+  private void doTestMultipleSnapshots(boolean saveNamespace)
+      throws IOException, AccessControlException {
+    Path path = new Path("/test");
+    doWriteAndAbort(fs, path);
+    fs.createSnapshot(path, "s2");
+    fs.delete(new Path("/test/test"), true);
+    fs.deleteSnapshot(path, "s2");
+    if (saveNamespace) {
+      NameNode nameNode = cluster.getNameNode();
+      NameNodeAdapter.enterSafeMode(nameNode, false);
+      NameNodeAdapter.saveNamespace(nameNode);
+      NameNodeAdapter.leaveSafeMode(nameNode);
+    }
+    cluster.restartNameNode(true);
+  }
+  
+  @Test
+  public void testOpenFilesWithRename() throws Exception {
+    Path path = new Path("/test");
+    doWriteAndAbort(fs, path);
+
+    // check for zero sized blocks
+    Path fileWithEmptyBlock = new Path("/test/test/test4");
+    fs.create(fileWithEmptyBlock);
+    NamenodeProtocols nameNodeRpc = cluster.getNameNodeRpc();
+    String clientName = fs.getClient().getClientName();
+    // create one empty block
+    nameNodeRpc.addBlock(fileWithEmptyBlock.toString(), clientName, null, null,
+        INodeId.GRANDFATHER_INODE_ID, null);
+    fs.createSnapshot(path, "s2");
+
+    fs.rename(new Path("/test/test"), new Path("/test/test-renamed"));
+    fs.delete(new Path("/test/test-renamed"), true);
+    NameNode nameNode = cluster.getNameNode();
+    NameNodeAdapter.enterSafeMode(nameNode, false);
+    NameNodeAdapter.saveNamespace(nameNode);
+    NameNodeAdapter.leaveSafeMode(nameNode);
+    cluster.restartNameNode(true);
+  }
 }
 }

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

@@ -100,6 +100,20 @@ Release 2.3.0 - UNRELEASED
     YARN-1222. Make improvements in ZKRMStateStore for fencing (Karthik
     YARN-1222. Make improvements in ZKRMStateStore for fencing (Karthik
     Kambatla via bikas)
     Kambatla via bikas)
 
 
+    YARN-709. Added tests to verify validity of delegation tokens and logging of
+    appsummary after RM restart. (Jian He via vinodkv)
+
+    YARN-1210. Changed RM to start new app-attempts on RM restart only after
+    ensuring that previous AM exited or after expiry time. (Omkar Vinit Joshi via
+    vinodkv)
+
+    YARN-674. Fixed ResourceManager to renew DelegationTokens on submission
+    asynchronously to work around potential slowness in state-store. (Omkar Vinit
+    Joshi via vinodkv)
+
+    YARN-584. In scheduler web UIs, queues unexpand on refresh. (Harshit
+    Daga via Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -139,6 +153,9 @@ Release 2.3.0 - UNRELEASED
     YARN-1411. HA config shouldn't affect NodeManager RPC addresses (Karthik
     YARN-1411. HA config shouldn't affect NodeManager RPC addresses (Karthik
     Kambatla via bikas)
     Kambatla via bikas)
 
 
+    YARN-1419. TestFifoScheduler.testAppAttemptMetrics fails intermittently
+    under jdk7 (Jonathan Eagles via jlowe)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -504,6 +504,11 @@ public class YarnConfiguration extends Configuration {
       RM_PREFIX + "delayed.delegation-token.removal-interval-ms";
       RM_PREFIX + "delayed.delegation-token.removal-interval-ms";
   public static final long DEFAULT_RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
   public static final long DEFAULT_RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
       30000l;
       30000l;
+  
+  /** Delegation Token renewer thread count */
+  public static final String RM_DELEGATION_TOKEN_RENEWER_THREAD_COUNT =
+      RM_PREFIX + "delegation-token-renewer.thread-count";
+  public static final int DEFAULT_RM_DELEGATION_TOKEN_RENEWER_THREAD_COUNT = 50;
 
 
   /** Whether to enable log aggregation */
   /** Whether to enable log aggregation */
   public static final String LOG_AGGREGATION_ENABLED = YARN_PREFIX
   public static final String LOG_AGGREGATION_ENABLED = YARN_PREFIX

+ 21 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java

@@ -20,15 +20,29 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 
 
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
+import org.apache.hadoop.yarn.util.Records;
 
 
-public interface NodeHeartbeatRequest {
+public abstract class NodeHeartbeatRequest {
+  
+  public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
+      MasterKey lastKnownContainerTokenMasterKey,
+      MasterKey lastKnownNMTokenMasterKey) {
+    NodeHeartbeatRequest nodeHeartbeatRequest =
+        Records.newRecord(NodeHeartbeatRequest.class);
+    nodeHeartbeatRequest.setNodeStatus(nodeStatus);
+    nodeHeartbeatRequest
+        .setLastKnownContainerTokenMasterKey(lastKnownContainerTokenMasterKey);
+    nodeHeartbeatRequest
+        .setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
+    return nodeHeartbeatRequest;
+  }
 
 
-  NodeStatus getNodeStatus();
-  void setNodeStatus(NodeStatus status);
+  public abstract NodeStatus getNodeStatus();
+  public abstract void setNodeStatus(NodeStatus status);
 
 
-  MasterKey getLastKnownContainerTokenMasterKey();
-  void setLastKnownContainerTokenMasterKey(MasterKey secretKey);
+  public abstract MasterKey getLastKnownContainerTokenMasterKey();
+  public abstract void setLastKnownContainerTokenMasterKey(MasterKey secretKey);
   
   
-  MasterKey getLastKnownNMTokenMasterKey();
-  void setLastKnownNMTokenMasterKey(MasterKey secretKey);
+  public abstract MasterKey getLastKnownNMTokenMasterKey();
+  public abstract void setLastKnownNMTokenMasterKey(MasterKey secretKey);
 }
 }

+ 29 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java

@@ -18,17 +18,37 @@
 
 
 package org.apache.hadoop.yarn.server.api.protocolrecords;
 package org.apache.hadoop.yarn.server.api.protocolrecords;
 
 
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
 
 
-public interface RegisterNodeManagerRequest {
-  NodeId getNodeId();
-  int getHttpPort();
-  Resource getResource();
-  String getNMVersion();
+public abstract class RegisterNodeManagerRequest {
+  
+  public static RegisterNodeManagerRequest newInstance(NodeId nodeId,
+      int httpPort, Resource resource, String nodeManagerVersionId,
+      List<ContainerStatus> containerStatuses) {
+    RegisterNodeManagerRequest request =
+        Records.newRecord(RegisterNodeManagerRequest.class);
+    request.setHttpPort(httpPort);
+    request.setResource(resource);
+    request.setNodeId(nodeId);
+    request.setNMVersion(nodeManagerVersionId);
+    request.setContainerStatuses(containerStatuses);
+    return request;
+  }
+  
+  public abstract NodeId getNodeId();
+  public abstract int getHttpPort();
+  public abstract Resource getResource();
+  public abstract String getNMVersion();
+  public abstract List<ContainerStatus> getContainerStatuses();
   
   
-  void setNodeId(NodeId nodeId);
-  void setHttpPort(int port);
-  void setResource(Resource resource);
-  void setNMVersion(String version);
+  public abstract void setNodeId(NodeId nodeId);
+  public abstract void setHttpPort(int port);
+  public abstract void setResource(Resource resource);
+  public abstract void setNMVersion(String version);
+  public abstract void setContainerStatuses(List<ContainerStatus> containerStatuses);
 }
 }

+ 16 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java

@@ -18,7 +18,6 @@
 
 
 package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
 
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
@@ -29,8 +28,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl;
 
 
-public class NodeHeartbeatRequestPBImpl extends
-    ProtoBase<NodeHeartbeatRequestProto> implements NodeHeartbeatRequest {
+public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   NodeHeartbeatRequestProto proto = NodeHeartbeatRequestProto.getDefaultInstance();
   NodeHeartbeatRequestProto proto = NodeHeartbeatRequestProto.getDefaultInstance();
   NodeHeartbeatRequestProto.Builder builder = null;
   NodeHeartbeatRequestProto.Builder builder = null;
   boolean viaProto = false;
   boolean viaProto = false;
@@ -55,6 +53,21 @@ public class NodeHeartbeatRequestPBImpl extends
     return proto;
     return proto;
   }
   }
 
 
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
   private void mergeLocalToBuilder() {
   private void mergeLocalToBuilder() {
     if (this.nodeStatus != null) {
     if (this.nodeStatus != null) {
       builder.setNodeStatus(convertToProtoFormat(this.nodeStatus));
       builder.setNodeStatus(convertToProtoFormat(this.nodeStatus));

+ 98 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerRequestPBImpl.java

@@ -19,11 +19,21 @@
 package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
 
 
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto;
@@ -32,13 +42,14 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
 
 
 
 
     
     
-public class RegisterNodeManagerRequestPBImpl extends ProtoBase<RegisterNodeManagerRequestProto> implements RegisterNodeManagerRequest {
+public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest {
   RegisterNodeManagerRequestProto proto = RegisterNodeManagerRequestProto.getDefaultInstance();
   RegisterNodeManagerRequestProto proto = RegisterNodeManagerRequestProto.getDefaultInstance();
   RegisterNodeManagerRequestProto.Builder builder = null;
   RegisterNodeManagerRequestProto.Builder builder = null;
   boolean viaProto = false;
   boolean viaProto = false;
   
   
   private Resource resource = null;
   private Resource resource = null;
   private NodeId nodeId = null;
   private NodeId nodeId = null;
+  private List<ContainerStatus> containerStatuses = null;
   
   
   public RegisterNodeManagerRequestPBImpl() {
   public RegisterNodeManagerRequestPBImpl() {
     builder = RegisterNodeManagerRequestProto.newBuilder();
     builder = RegisterNodeManagerRequestProto.newBuilder();
@@ -57,6 +68,9 @@ public class RegisterNodeManagerRequestPBImpl extends ProtoBase<RegisterNodeMana
   }
   }
 
 
   private void mergeLocalToBuilder() {
   private void mergeLocalToBuilder() {
+    if (this.containerStatuses != null) {
+      addContainerStatusesToProto();
+    }
     if (this.resource != null) {
     if (this.resource != null) {
       builder.setResource(convertToProtoFormat(this.resource));
       builder.setResource(convertToProtoFormat(this.resource));
     }
     }
@@ -139,6 +153,81 @@ public class RegisterNodeManagerRequestPBImpl extends ProtoBase<RegisterNodeMana
     builder.setHttpPort(httpPort);
     builder.setHttpPort(httpPort);
   }
   }
 
 
+  @Override
+  public List<ContainerStatus> getContainerStatuses() {
+    initContainerStatuses();
+    return containerStatuses;
+  }
+  
+  private void initContainerStatuses() {
+    if (this.containerStatuses != null) {
+      return;
+    }
+    RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerStatusProto> list = p.getContainerStatusesList();
+    this.containerStatuses = new ArrayList<ContainerStatus>();
+    for (ContainerStatusProto c : list) {
+      this.containerStatuses.add(convertFromProtoFormat(c));
+    }
+  }
+
+  @Override
+  public void setContainerStatuses(List<ContainerStatus> containers) {
+    if (containers == null) {
+      return;
+    }
+    initContainerStatuses();
+    this.containerStatuses.addAll(containers);
+  }
+  
+  private void addContainerStatusesToProto() {
+    maybeInitBuilder();
+    builder.clearContainerStatuses();
+    if (containerStatuses == null) {
+      return;
+    }
+    Iterable<ContainerStatusProto> it = new Iterable<ContainerStatusProto>() {
+      
+      @Override
+      public Iterator<ContainerStatusProto> iterator() {
+        return new Iterator<ContainerStatusProto>() {
+          Iterator<ContainerStatus> iter = containerStatuses.iterator();
+          
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+          
+          @Override
+          public ContainerStatusProto next() {
+            return convertToProtoFormat(iter.next());  
+          }
+          
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllContainerStatuses(it);
+  }
+  
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+  
   @Override
   @Override
   public String getNMVersion() {
   public String getNMVersion() {
     RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
     RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
@@ -170,6 +259,11 @@ public class RegisterNodeManagerRequestPBImpl extends ProtoBase<RegisterNodeMana
     return ((ResourcePBImpl)t).getProto();
     return ((ResourcePBImpl)t).getProto();
   }
   }
 
 
-
-
-}  
+  private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto c) {
+    return new ContainerStatusPBImpl(c);
+  }
+  
+  private ContainerStatusProto convertToProtoFormat(ContainerStatus c) {
+    return ((ContainerStatusPBImpl)c).getProto();
+  }
+}

+ 17 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java

@@ -22,10 +22,24 @@ import java.util.List;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.util.Records;
 
 
 
 
-public interface NodeStatus {
+public abstract class NodeStatus {
   
   
+  public static NodeStatus newInstance(NodeId nodeId, int responseId,
+      List<ContainerStatus> containerStatuses,
+      List<ApplicationId> keepAliveApplications,
+      NodeHealthStatus nodeHealthStatus) {
+    NodeStatus nodeStatus = Records.newRecord(NodeStatus.class);
+    nodeStatus.setResponseId(responseId);
+    nodeStatus.setNodeId(nodeId);
+    nodeStatus.setContainersStatuses(containerStatuses);
+    nodeStatus.setKeepAliveApplications(keepAliveApplications);
+    nodeStatus.setNodeHealthStatus(nodeHealthStatus);
+    return nodeStatus;
+  }
+
   public abstract NodeId getNodeId();
   public abstract NodeId getNodeId();
   public abstract int getResponseId();
   public abstract int getResponseId();
   
   
@@ -36,8 +50,8 @@ public interface NodeStatus {
   public abstract List<ApplicationId> getKeepAliveApplications();
   public abstract List<ApplicationId> getKeepAliveApplications();
   public abstract void setKeepAliveApplications(List<ApplicationId> appIds);
   public abstract void setKeepAliveApplications(List<ApplicationId> appIds);
   
   
-  NodeHealthStatus getNodeHealthStatus();
-  void setNodeHealthStatus(NodeHealthStatus healthStatus);
+  public abstract NodeHealthStatus getNodeHealthStatus();
+  public abstract void setNodeHealthStatus(NodeHealthStatus healthStatus);
 
 
   public abstract void setNodeId(NodeId nodeId);
   public abstract void setNodeId(NodeId nodeId);
   public abstract void setResponseId(int responseId);
   public abstract void setResponseId(int responseId);

+ 16 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
@@ -40,8 +39,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
     
     
 
 
-public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements
-    NodeStatus {
+public class NodeStatusPBImpl extends NodeStatus {
   NodeStatusProto proto = NodeStatusProto.getDefaultInstance();
   NodeStatusProto proto = NodeStatusProto.getDefaultInstance();
   NodeStatusProto.Builder builder = null;
   NodeStatusProto.Builder builder = null;
   boolean viaProto = false;
   boolean viaProto = false;
@@ -166,6 +164,21 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements
     builder.addAllKeepAliveApplications(iterable);
     builder.addAllKeepAliveApplications(iterable);
   }
   }
 
 
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
   @Override
   @Override
   public synchronized int getResponseId() {
   public synchronized int getResponseId() {
     NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
     NodeStatusProtoOrBuilder p = viaProto ? proto : builder;

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -30,6 +30,7 @@ message RegisterNodeManagerRequestProto {
   optional int32 http_port = 3;
   optional int32 http_port = 3;
   optional ResourceProto resource = 4;
   optional ResourceProto resource = 4;
   optional string nm_version = 5;
   optional string nm_version = 5;
+  repeated ContainerStatusProto containerStatuses = 6;
 }
 }
 
 
 message RegisterNodeManagerResponseProto {
 message RegisterNodeManagerResponseProto {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java

@@ -26,7 +26,7 @@ public interface NodeStatusUpdater extends Service {
 
 
   void sendOutofBandHeartBeat();
   void sendOutofBandHeartBeat();
 
 
-  NodeStatus getNodeStatusAndUpdateContainersInContext();
+  NodeStatus getNodeStatusAndUpdateContainersInContext(int responseId);
 
 
   long getRMIdentifier();
   long getRMIdentifier();
   
   

+ 46 - 50
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -48,8 +48,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 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.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.ServerRMProxy;
 import org.apache.hadoop.yarn.server.api.ServerRMProxy;
@@ -89,7 +87,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   private String nodeManagerVersionId;
   private String nodeManagerVersionId;
   private String minimumResourceManagerVersion;
   private String minimumResourceManagerVersion;
   private volatile boolean isStopped;
   private volatile boolean isStopped;
-  private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private boolean tokenKeepAliveEnabled;
   private boolean tokenKeepAliveEnabled;
   private long tokenRemovalDelayMs;
   private long tokenRemovalDelayMs;
   /** Keeps track of when the next keep alive request should be sent for an app*/
   /** Keeps track of when the next keep alive request should be sent for an app*/
@@ -134,9 +131,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         conf.getInt(
         conf.getInt(
             YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES);
             YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES);
 
 
-    this.totalResource = recordFactory.newRecordInstance(Resource.class);
-    this.totalResource.setMemory(memoryMb);
-    this.totalResource.setVirtualCores(virtualCores);
+    this.totalResource = Resource.newInstance(memoryMb, virtualCores);
     metrics.addResource(totalResource);
     metrics.addResource(totalResource);
     this.tokenKeepAliveEnabled = isTokenKeepAliveEnabled(conf);
     this.tokenKeepAliveEnabled = isTokenKeepAliveEnabled(conf);
     this.tokenRemovalDelayMs =
     this.tokenRemovalDelayMs =
@@ -238,13 +233,17 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  protected void registerWithRM() throws YarnException, IOException {
+  protected void registerWithRM()
+      throws YarnException, IOException {
+    List<ContainerStatus> containerStatuses =
+        this.updateAndGetContainerStatuses();
     RegisterNodeManagerRequest request =
     RegisterNodeManagerRequest request =
-        recordFactory.newRecordInstance(RegisterNodeManagerRequest.class);
-    request.setHttpPort(this.httpPort);
-    request.setResource(this.totalResource);
-    request.setNodeId(this.nodeId);
-    request.setNMVersion(this.nodeManagerVersionId);
+        RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource,
+          nodeManagerVersionId, containerStatuses);
+    if (containerStatuses != null) {
+      LOG.info("Registering with RM using finished containers :"
+          + containerStatuses);
+    }
     RegisterNodeManagerResponse regNMResponse =
     RegisterNodeManagerResponse regNMResponse =
         resourceTracker.registerNodeManager(request);
         resourceTracker.registerNodeManager(request);
     this.rmIdentifier = regNMResponse.getRMIdentifier();
     this.rmIdentifier = regNMResponse.getRMIdentifier();
@@ -323,13 +322,33 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   }
   }
 
 
   @Override
   @Override
-  public NodeStatus getNodeStatusAndUpdateContainersInContext() {
+  public NodeStatus getNodeStatusAndUpdateContainersInContext(
+      int responseId) {
 
 
-    NodeStatus nodeStatus = recordFactory.newRecordInstance(NodeStatus.class);
-    nodeStatus.setNodeId(this.nodeId);
+    NodeHealthStatus nodeHealthStatus = this.context.getNodeHealthStatus();
+    nodeHealthStatus.setHealthReport(healthChecker.getHealthReport());
+    nodeHealthStatus.setIsNodeHealthy(healthChecker.isHealthy());
+    nodeHealthStatus.setLastHealthReportTime(
+        healthChecker.getLastHealthReportTime());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Node's health-status : " + nodeHealthStatus.getIsNodeHealthy()
+                + ", " + nodeHealthStatus.getHealthReport());
+    }
+    List<ContainerStatus> containersStatuses = updateAndGetContainerStatuses();
+    LOG.debug(this.nodeId + " sending out status for "
+        + containersStatuses.size() + " containers");
+    NodeStatus nodeStatus = NodeStatus.newInstance(nodeId, responseId,
+      containersStatuses, createKeepAliveApplicationList(), nodeHealthStatus);
 
 
-    int numActiveContainers = 0;
-    List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
+    return nodeStatus;
+  }
+
+  /*
+   * It will return current container statuses. If any container has
+   * COMPLETED then it will be removed from context. 
+   */
+  private List<ContainerStatus> updateAndGetContainerStatuses() {
+    List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
     for (Iterator<Entry<ContainerId, Container>> i =
     for (Iterator<Entry<ContainerId, Container>> i =
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
       Entry<ContainerId, Container> e = i.next();
       Entry<ContainerId, Container> e = i.next();
@@ -339,8 +358,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       // Clone the container to send it to the RM
       // Clone the container to send it to the RM
       org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus = 
       org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus = 
           container.cloneAndGetContainerStatus();
           container.cloneAndGetContainerStatus();
-      containersStatuses.add(containerStatus);
-      ++numActiveContainers;
+      containerStatuses.add(containerStatus);
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Sending out status for container: " + containerStatus);
         LOG.debug("Sending out status for container: " + containerStatus);
       }
       }
@@ -356,26 +374,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         LOG.info("Removed completed container " + containerId);
         LOG.info("Removed completed container " + containerId);
       }
       }
     }
     }
-    nodeStatus.setContainersStatuses(containersStatuses);
-
-    LOG.debug(this.nodeId + " sending out status for "
-        + numActiveContainers + " containers");
-
-    NodeHealthStatus nodeHealthStatus = this.context.getNodeHealthStatus();
-    nodeHealthStatus.setHealthReport(healthChecker.getHealthReport());
-    nodeHealthStatus.setIsNodeHealthy(healthChecker.isHealthy());
-    nodeHealthStatus.setLastHealthReportTime(
-        healthChecker.getLastHealthReportTime());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Node's health-status : " + nodeHealthStatus.getIsNodeHealthy()
-                + ", " + nodeHealthStatus.getHealthReport());
-    }
-    nodeStatus.setNodeHealthStatus(nodeHealthStatus);
-
-    List<ApplicationId> keepAliveAppIds = createKeepAliveApplicationList();
-    nodeStatus.setKeepAliveApplications(keepAliveAppIds);
-    
-    return nodeStatus;
+    return containerStatuses;
   }
   }
 
 
   private void trackAppsForKeepAlive(List<ApplicationId> appIds) {
   private void trackAppsForKeepAlive(List<ApplicationId> appIds) {
@@ -458,18 +457,15 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
           // Send heartbeat
           // Send heartbeat
           try {
           try {
             NodeHeartbeatResponse response = null;
             NodeHeartbeatResponse response = null;
-            NodeStatus nodeStatus = getNodeStatusAndUpdateContainersInContext();
-            nodeStatus.setResponseId(lastHeartBeatID);
+            NodeStatus nodeStatus =
+                getNodeStatusAndUpdateContainersInContext(lastHeartBeatID);
             
             
-            NodeHeartbeatRequest request = recordFactory
-                .newRecordInstance(NodeHeartbeatRequest.class);
-            request.setNodeStatus(nodeStatus);
-            request
-              .setLastKnownContainerTokenMasterKey(NodeStatusUpdaterImpl.this.context
-                .getContainerTokenSecretManager().getCurrentKey());
-            request
-              .setLastKnownNMTokenMasterKey(NodeStatusUpdaterImpl.this.context
-                .getNMTokenSecretManager().getCurrentKey());
+            NodeHeartbeatRequest request =
+                NodeHeartbeatRequest.newInstance(nodeStatus,
+                  NodeStatusUpdaterImpl.this.context
+                    .getContainerTokenSecretManager().getCurrentKey(),
+                  NodeStatusUpdaterImpl.this.context.getNMTokenSecretManager()
+                    .getCurrentKey());
             response = resourceTracker.nodeHeartbeat(request);
             response = resourceTracker.nodeHeartbeat(request);
             //get next heartbeat interval from response
             //get next heartbeat interval from response
             nextHeartBeatInterval = response.getNextHeartBeatInterval();
             nextHeartBeatInterval = response.getNextHeartBeatInterval();

+ 24 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -28,6 +28,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -64,6 +65,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.SerializedException;
@@ -371,17 +373,31 @@ public class ContainerManagerImpl extends CompositeService implements
 
 
     this.handle(new CMgrCompletedContainersEvent(containerIds,
     this.handle(new CMgrCompletedContainersEvent(containerIds,
       CMgrCompletedContainersEvent.Reason.ON_NODEMANAGER_RESYNC));
       CMgrCompletedContainersEvent.Reason.ON_NODEMANAGER_RESYNC));
-    while (!containers.isEmpty()) {
-      try {
-        Thread.sleep(1000);
-        nodeStatusUpdater.getNodeStatusAndUpdateContainersInContext();
-      } catch (InterruptedException ex) {
-        LOG.warn("Interrupted while sleeping on container kill on resync", ex);
+
+    /*
+     * We will wait till all the containers change their state to COMPLETE. We
+     * will not remove the container statuses from nm context because these
+     * are used while re-registering node manager with resource manager.
+     */
+    boolean allContainersCompleted = false;
+    while (!containers.isEmpty() && !allContainersCompleted) {
+      allContainersCompleted = true;
+      for (Entry<ContainerId, Container> container : containers.entrySet()) {
+        if (((ContainerImpl) container.getValue()).getCurrentState()
+            != ContainerState.COMPLETE) {
+          allContainersCompleted = false;
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ex) {
+            LOG.warn("Interrupted while sleeping on container kill on resync",
+              ex);
+          }
+          break;
+        }
       }
       }
     }
     }
-
     // All containers killed
     // All containers killed
-    if (containers.isEmpty()) {
+    if (allContainersCompleted) {
       LOG.info("All containers in DONE state");
       LOG.info("All containers in DONE state");
     } else {
     } else {
       LOG.info("Done waiting for containers to be killed. Still alive: " +
       LOG.info("Done waiting for containers to be killed. Still alive: " +

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -302,7 +302,7 @@ public class ContainerImpl implements Container {
   private final StateMachine<ContainerState, ContainerEventType, ContainerEvent>
   private final StateMachine<ContainerState, ContainerEventType, ContainerEvent>
     stateMachine;
     stateMachine;
 
 
-  private org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
+  public org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
     switch (stateMachine.getCurrentState()) {
     switch (stateMachine.getCurrentState()) {
     case NEW:
     case NEW:
     case LOCALIZING:
     case LOCALIZING:

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -318,7 +318,7 @@ public class ClientRMService extends AbstractService implements
     try {
     try {
       // call RMAppManager to submit application directly
       // call RMAppManager to submit application directly
       rmAppManager.submitApplication(submissionContext,
       rmAppManager.submitApplication(submissionContext,
-          System.currentTimeMillis(), false, user);
+          System.currentTimeMillis(), user, false, null);
 
 
       LOG.info("Application with id " + applicationId.getId() + 
       LOG.info("Application with id " + applicationId.getId() + 
           " submitted by user " + user);
           " submitted by user " + user);

+ 96 - 52
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
@@ -54,6 +55,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
 /**
  * This class manages the list of applications for the resource manager. 
  * This class manages the list of applications for the resource manager. 
  */
  */
@@ -165,6 +168,11 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     }
     }
   }
   }
 
 
+  @VisibleForTesting
+  public void logApplicationSummary(ApplicationId appId) {
+    ApplicationSummary.logAppSummary(rmContext.getRMApps().get(appId));
+  }
+
   protected synchronized void setCompletedAppsMax(int max) {
   protected synchronized void setCompletedAppsMax(int max) {
     this.completedAppsMax = max;
     this.completedAppsMax = max;
   }
   }
@@ -229,35 +237,63 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       this.applicationACLsManager.removeApplication(removeId);
       this.applicationACLsManager.removeApplication(removeId);
     }
     }
   }
   }
-
+  
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   protected void submitApplication(
   protected void submitApplication(
       ApplicationSubmissionContext submissionContext, long submitTime,
       ApplicationSubmissionContext submissionContext, long submitTime,
-      boolean isRecovered, String user) throws YarnException {
+      String user, boolean isRecovered, RMState state) throws YarnException {
     ApplicationId applicationId = submissionContext.getApplicationId();
     ApplicationId applicationId = submissionContext.getApplicationId();
 
 
-    // Validation of the ApplicationSubmissionContext needs to be completed
-    // here. Only those fields that are dependent on RM's configuration are
-    // checked here as they have to be validated whether they are part of new
-    // submission or just being recovered.
+    RMAppImpl application =
+        createAndPopulateNewRMApp(submissionContext, submitTime, user);
 
 
-    // Check whether AM resource requirements are within required limits
-    if (!submissionContext.getUnmanagedAM()) {
-      ResourceRequest amReq = BuilderUtils.newResourceRequest(
-          RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
-          submissionContext.getResource(), 1);
+    if (isRecovered) {
+      recoverApplication(state, application);
+      RMAppState rmAppState =
+          state.getApplicationState().get(applicationId).getState();
+      if (isApplicationInFinalState(rmAppState)) {
+        // We are synchronously moving the application into final state so that
+        // momentarily client will not see this application in NEW state. Also
+        // for finished applications we will avoid renewing tokens.
+        application
+            .handle(new RMAppEvent(applicationId, RMAppEventType.RECOVER));
+        return;
+      }
+    }
+    
+    if (UserGroupInformation.isSecurityEnabled()) {
+      Credentials credentials = null;
       try {
       try {
-        SchedulerUtils.validateResourceRequest(amReq,
-            scheduler.getMaximumResourceCapability());
-      } catch (InvalidResourceRequestException e) {
-        LOG.warn("RM app submission failed in validating AM resource request"
-            + " for application " + applicationId, e);
-        throw e;
+        credentials = parseCredentials(submissionContext);
+      } catch (Exception e) {
+        LOG.warn(
+            "Unable to parse credentials.", e);
+        // Sending APP_REJECTED is fine, since we assume that the
+        // RMApp is in NEW state and thus we haven't yet informed the
+        // scheduler about the existence of the application
+        assert application.getState() == RMAppState.NEW;
+        this.rmContext.getDispatcher().getEventHandler().handle(
+            new RMAppRejectedEvent(applicationId, e.getMessage()));
+        throw RPCUtil.getRemoteException(e);
       }
       }
+      this.rmContext.getDelegationTokenRenewer().addApplication(
+          applicationId, credentials,
+          submissionContext.getCancelTokensWhenComplete(), isRecovered);
+    } else {
+      this.rmContext.getDispatcher().getEventHandler()
+          .handle(new RMAppEvent(applicationId,
+              isRecovered ? RMAppEventType.RECOVER : RMAppEventType.START));
     }
     }
+  }
 
 
+  private RMAppImpl createAndPopulateNewRMApp(
+      ApplicationSubmissionContext submissionContext,
+      long submitTime, String user)
+      throws YarnException {
+    ApplicationId applicationId = submissionContext.getApplicationId();
+    validateResourceRequest(submissionContext);
     // Create RMApp
     // Create RMApp
-    RMApp application =
+    RMAppImpl application =
         new RMAppImpl(applicationId, rmContext, this.conf,
         new RMAppImpl(applicationId, rmContext, this.conf,
             submissionContext.getApplicationName(), user,
             submissionContext.getApplicationName(), user,
             submissionContext.getQueue(),
             submissionContext.getQueue(),
@@ -274,35 +310,52 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       LOG.warn(message);
       LOG.warn(message);
       throw RPCUtil.getRemoteException(message);
       throw RPCUtil.getRemoteException(message);
     }
     }
-
     // Inform the ACLs Manager
     // Inform the ACLs Manager
     this.applicationACLsManager.addApplication(applicationId,
     this.applicationACLsManager.addApplication(applicationId,
         submissionContext.getAMContainerSpec().getApplicationACLs());
         submissionContext.getAMContainerSpec().getApplicationACLs());
+    return application;
+  }
 
 
-    try {
-      // Setup tokens for renewal
-      if (UserGroupInformation.isSecurityEnabled()) {
-        this.rmContext.getDelegationTokenRenewer().addApplication(
-            applicationId,parseCredentials(submissionContext),
-            submissionContext.getCancelTokensWhenComplete()
-            );
+  private void validateResourceRequest(
+      ApplicationSubmissionContext submissionContext)
+      throws InvalidResourceRequestException {
+    // Validation of the ApplicationSubmissionContext needs to be completed
+    // here. Only those fields that are dependent on RM's configuration are
+    // checked here as they have to be validated whether they are part of new
+    // submission or just being recovered.
+
+    // Check whether AM resource requirements are within required limits
+    if (!submissionContext.getUnmanagedAM()) {
+      ResourceRequest amReq = BuilderUtils.newResourceRequest(
+          RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+          submissionContext.getResource(), 1);
+      try {
+        SchedulerUtils.validateResourceRequest(amReq,
+            scheduler.getMaximumResourceCapability());
+      } catch (InvalidResourceRequestException e) {
+        LOG.warn("RM app submission failed in validating AM resource request"
+            + " for application " + submissionContext.getApplicationId(), e);
+        throw e;
       }
       }
-    } catch (IOException ie) {
-      LOG.warn(
-          "Unable to add the application to the delegation token renewer.",
-          ie);
-      // Sending APP_REJECTED is fine, since we assume that the
-      // RMApp is in NEW state and thus we havne't yet informed the
-      // Scheduler about the existence of the application
-      this.rmContext.getDispatcher().getEventHandler().handle(
-          new RMAppRejectedEvent(applicationId, ie.getMessage()));
-      throw RPCUtil.getRemoteException(ie);
     }
     }
+  }
 
 
-    if (!isRecovered) {
-      // All done, start the RMApp
-      this.rmContext.getDispatcher().getEventHandler()
-        .handle(new RMAppEvent(applicationId, RMAppEventType.START));
+  private void recoverApplication(RMState state, RMAppImpl application)
+      throws YarnException {
+    try {
+      application.recover(state);
+    } catch (Exception e) {
+      LOG.error("Error recovering application", e);
+      throw new YarnException(e);
+    }
+  }
+
+  private boolean isApplicationInFinalState(RMAppState rmAppState) {
+    if (rmAppState == RMAppState.FINISHED || rmAppState == RMAppState.FAILED
+        || rmAppState == RMAppState.KILLED) {
+      return true;
+    } else {
+      return false;
     }
     }
   }
   }
   
   
@@ -328,17 +381,9 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     LOG.info("Recovering " + appStates.size() + " applications");
     LOG.info("Recovering " + appStates.size() + " applications");
     for (ApplicationState appState : appStates.values()) {
     for (ApplicationState appState : appStates.values()) {
       LOG.info("Recovering application " + appState.getAppId());
       LOG.info("Recovering application " + appState.getAppId());
+      
       submitApplication(appState.getApplicationSubmissionContext(),
       submitApplication(appState.getApplicationSubmissionContext(),
-        appState.getSubmitTime(), true, appState.getUser());
-      // re-populate attempt information in application
-      RMAppImpl appImpl =
-          (RMAppImpl) rmContext.getRMApps().get(appState.getAppId());
-      appImpl.recover(state);
-      // Recover the app synchronously, as otherwise client is possible to see
-      // the application not recovered before it is actually recovered because
-      // ClientRMService is already started at this point of time.
-      appImpl.handle(new RMAppEvent(appImpl.getApplicationId(),
-        RMAppEventType.RECOVER));
+        appState.getSubmitTime(), appState.getUser(), true, state);
     }
     }
   }
   }
 
 
@@ -351,8 +396,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       case APP_COMPLETED: 
       case APP_COMPLETED: 
       {
       {
         finishApplication(applicationId);
         finishApplication(applicationId);
-        ApplicationSummary.logAppSummary(
-            rmContext.getRMApps().get(applicationId));
+        logApplicationSummary(applicationId);
         checkAppNumCompletedLimit(); 
         checkAppNumCompletedLimit(); 
       } 
       } 
       break;
       break;

+ 35 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -29,6 +29,10 @@ import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.VersionUtil;
 import org.apache.hadoop.util.VersionUtil;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
@@ -46,14 +50,17 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.RackResolver;
@@ -183,6 +190,33 @@ public class ResourceTrackerService extends AbstractService implements
     Resource capability = request.getResource();
     Resource capability = request.getResource();
     String nodeManagerVersion = request.getNMVersion();
     String nodeManagerVersion = request.getNMVersion();
 
 
+    if (!request.getContainerStatuses().isEmpty()) {
+      LOG.info("received container statuses on node manager register :"
+          + request.getContainerStatuses());
+      for (ContainerStatus containerStatus : request.getContainerStatuses()) {
+        ApplicationAttemptId appAttemptId =
+            containerStatus.getContainerId().getApplicationAttemptId();
+        RMApp rmApp =
+            rmContext.getRMApps().get(appAttemptId.getApplicationId());
+        if (rmApp != null) {
+          RMAppAttempt rmAppAttempt = rmApp.getRMAppAttempt(appAttemptId);
+          if (rmAppAttempt.getMasterContainer().getId()
+              .equals(containerStatus.getContainerId())
+              && containerStatus.getState() == ContainerState.COMPLETE) {
+            // sending master container finished event.
+            RMAppAttemptContainerFinishedEvent evt =
+                new RMAppAttemptContainerFinishedEvent(appAttemptId,
+                    containerStatus);
+            rmContext.getDispatcher().getEventHandler().handle(evt);
+          }
+        } else {
+          LOG.error("Received finished container :"
+              + containerStatus.getContainerId()
+              + " for non existing application :"
+              + appAttemptId.getApplicationId());
+        }
+      }
+    }
     RegisterNodeManagerResponse response = recordFactory
     RegisterNodeManagerResponse response = recordFactory
         .newRecordInstance(RegisterNodeManagerResponse.class);
         .newRecordInstance(RegisterNodeManagerResponse.class);
 
 

+ 33 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -132,8 +132,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
     .addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
         RMAppEventType.START, new RMAppSavingTransition())
         RMAppEventType.START, new RMAppSavingTransition())
     .addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
     .addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
-            RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED,
-            RMAppState.FINAL_SAVING),
+            RMAppState.RUNNING, RMAppState.FINISHED, RMAppState.FAILED,
+            RMAppState.KILLED, RMAppState.FINAL_SAVING),
         RMAppEventType.RECOVER, new RMAppRecoveredTransition())
         RMAppEventType.RECOVER, new RMAppRecoveredTransition())
     .addTransition(RMAppState.NEW, RMAppState.FINAL_SAVING, RMAppEventType.KILL,
     .addTransition(RMAppState.NEW, RMAppState.FINAL_SAVING, RMAppEventType.KILL,
         new FinalSavingTransition(
         new FinalSavingTransition(
@@ -611,11 +611,11 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.diagnostics.append(appState.getDiagnostics());
     this.diagnostics.append(appState.getDiagnostics());
     this.storedFinishTime = appState.getFinishTime();
     this.storedFinishTime = appState.getFinishTime();
     this.startTime = appState.getStartTime();
     this.startTime = appState.getStartTime();
+
     for(int i=0; i<appState.getAttemptCount(); ++i) {
     for(int i=0; i<appState.getAttemptCount(); ++i) {
       // create attempt
       // create attempt
       createNewAttempt(false);
       createNewAttempt(false);
-      // recover attempt
-      ((RMAppAttemptImpl) currentAttempt).recover(state);
+      ((RMAppAttemptImpl)this.currentAttempt).recover(state);
     }
     }
   }
   }
 
 
@@ -656,30 +656,35 @@ public class RMAppImpl implements RMApp, Recoverable {
     };
     };
   }
   }
 
 
+  @SuppressWarnings("unchecked")
   private static final class RMAppRecoveredTransition implements
   private static final class RMAppRecoveredTransition implements
       MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
       MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
+    
     @Override
     @Override
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
 
 
-      if (app.recoveredFinalState != null) {
-        FINAL_TRANSITION.transition(app, event);
-        return app.recoveredFinalState;
+      if (app.attempts.isEmpty()) {
+        // Saved application was not running any attempts.
+        app.createNewAttempt(true);
+        return RMAppState.SUBMITTED;        
+      } else {
+        /*
+         * If last attempt recovered final state is null .. it means attempt
+         * was started but AM container may or may not have started / finished.
+         * Therefore we should wait for it to finish.
+         */
+        for (RMAppAttempt attempt : app.getAppAttempts().values()) {
+          app.dispatcher.getEventHandler().handle(
+              new RMAppAttemptEvent(attempt.getAppAttemptId(),
+                  RMAppAttemptEventType.RECOVER));
+        }        
+        if (app.recoveredFinalState != null) {
+          FINAL_TRANSITION.transition(app, event);
+          return app.recoveredFinalState;
+        } else {
+          return RMAppState.RUNNING;
+        }
       }
       }
-      // Directly call AttemptFailedTransition, since now we deem that an
-      // application fails because of RM restart as a normal AM failure.
-
-      // Do not recover unmanaged applications since current recovery 
-      // mechanism of restarting attempts does not work for them.
-      // This will need to be changed in work preserving recovery in which 
-      // RM will re-connect with the running AM's instead of restarting them
-
-      // In work-preserve restart, if attemptCount == maxAttempts, the job still
-      // needs to be recovered because the last attempt may still be running.
-
-      // As part of YARN-1210, we may return ACCECPTED state waiting for AM to
-      // reregister or fail and remove the following code.
-      return new AttemptFailedTransition(RMAppState.SUBMITTED).transition(app,
-        event);
     }
     }
   }
   }
 
 
@@ -1017,4 +1022,10 @@ public class RMAppImpl implements RMApp, Recoverable {
       throw new YarnRuntimeException("Unknown state passed!");
       throw new YarnRuntimeException("Unknown state passed!");
     }
     }
   }
   }
+  
+  public static boolean isAppInFinalState(RMApp rmApp) {
+    RMAppState appState = rmApp.getState();
+    return appState == RMAppState.FAILED || appState == RMAppState.FINISHED
+        || appState == RMAppState.KILLED;
+  }
 }
 }

+ 32 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -68,11 +68,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.Appli
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
@@ -179,7 +182,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             new UnexpectedAMRegisteredTransition(), RMAppAttemptState.FAILED))
             new UnexpectedAMRegisteredTransition(), RMAppAttemptState.FAILED))
       .addTransition( RMAppAttemptState.NEW,
       .addTransition( RMAppAttemptState.NEW,
           EnumSet.of(RMAppAttemptState.FINISHED, RMAppAttemptState.KILLED,
           EnumSet.of(RMAppAttemptState.FINISHED, RMAppAttemptState.KILLED,
-            RMAppAttemptState.FAILED, RMAppAttemptState.RECOVERED),
+            RMAppAttemptState.FAILED, RMAppAttemptState.LAUNCHED),
           RMAppAttemptEventType.RECOVER, new AttemptRecoveredTransition())
           RMAppAttemptEventType.RECOVER, new AttemptRecoveredTransition())
           
           
       // Transitions from SUBMITTED state
       // Transitions from SUBMITTED state
@@ -386,25 +389,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
               RMAppAttemptEventType.UNREGISTERED,
               RMAppAttemptEventType.UNREGISTERED,
               RMAppAttemptEventType.KILL,
               RMAppAttemptEventType.KILL,
               RMAppAttemptEventType.STATUS_UPDATE))
               RMAppAttemptEventType.STATUS_UPDATE))
-              
-      // Transitions from RECOVERED State
-      .addTransition(
-          RMAppAttemptState.RECOVERED,
-          RMAppAttemptState.RECOVERED,
-          EnumSet.of(RMAppAttemptEventType.START,
-              RMAppAttemptEventType.APP_ACCEPTED,
-              RMAppAttemptEventType.APP_REJECTED,
-              RMAppAttemptEventType.EXPIRE,
-              RMAppAttemptEventType.LAUNCHED,
-              RMAppAttemptEventType.LAUNCH_FAILED,
-              RMAppAttemptEventType.REGISTERED,
-              RMAppAttemptEventType.CONTAINER_ALLOCATED,
-              RMAppAttemptEventType.CONTAINER_ACQUIRED,
-              RMAppAttemptEventType.ATTEMPT_NEW_SAVED,
-              RMAppAttemptEventType.CONTAINER_FINISHED,
-              RMAppAttemptEventType.UNREGISTERED,
-              RMAppAttemptEventType.KILL,
-              RMAppAttemptEventType.STATUS_UPDATE))
     .installTopology();
     .installTopology();
 
 
   public RMAppAttemptImpl(ApplicationAttemptId appAttemptId,
   public RMAppAttemptImpl(ApplicationAttemptId appAttemptId,
@@ -694,8 +678,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.proxiedTrackingUrl = generateProxyUriWithScheme(originalTrackingUrl);
     this.proxiedTrackingUrl = generateProxyUriWithScheme(originalTrackingUrl);
     this.finalStatus = attemptState.getFinalApplicationStatus();
     this.finalStatus = attemptState.getFinalApplicationStatus();
     this.startTime = attemptState.getStartTime();
     this.startTime = attemptState.getStartTime();
-    handle(new RMAppAttemptEvent(getAppAttemptId(),
-      RMAppAttemptEventType.RECOVER));
   }
   }
 
 
   private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
   private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
@@ -865,11 +847,38 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     @Override
     @Override
     public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
     public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
         RMAppAttemptEvent event) {
+      LOG.info("Recovering attempt :  recoverdFinalState :"
+          + appAttempt.recoveredFinalState);
       if (appAttempt.recoveredFinalState != null) {
       if (appAttempt.recoveredFinalState != null) {
         appAttempt.progress = 1.0f;
         appAttempt.progress = 1.0f;
+        RMApp rmApp =appAttempt.rmContext.getRMApps().get(
+            appAttempt.getAppAttemptId().getApplicationId());
+        // We will replay the final attempt only if last attempt is in final
+        // state but application is not in final state.
+        if (rmApp.getCurrentAppAttempt() == appAttempt
+            && !RMAppImpl.isAppInFinalState(rmApp)) {
+          (new BaseFinalTransition(appAttempt.recoveredFinalState)).transition(
+              appAttempt, event);
+        }
         return appAttempt.recoveredFinalState;
         return appAttempt.recoveredFinalState;
       } else {
       } else {
-        return RMAppAttemptState.RECOVERED;
+        /*
+         * Since the application attempt's final state is not saved that means
+         * for AM container (previous attempt) state must be one of these.
+         * 1) AM container may not have been launched (RM failed right before
+         * this).
+         * 2) AM container was successfully launched but may or may not have
+         * registered / unregistered.
+         * In whichever case we will wait (by moving attempt into LAUNCHED
+         * state) and mark this attempt failed (assuming non work preserving
+         * restart) only after
+         * 1) Node manager during re-registration heart beats back saying
+         * am container finished.
+         * 2) OR AMLivelinessMonitor expires this attempt (when am doesn't
+         * heart beat back).  
+         */
+        (new AMLaunchedTransition()).transition(appAttempt, event);
+        return RMAppAttemptState.LAUNCHED;
       }
       }
     }
     }
   }
   }

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptState.java

@@ -20,6 +20,5 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 
 public enum RMAppAttemptState {
 public enum RMAppAttemptState {
   NEW, SUBMITTED, SCHEDULED, ALLOCATED, LAUNCHED, FAILED, RUNNING, FINISHING, 
   NEW, SUBMITTED, SCHEDULED, ALLOCATED, LAUNCHED, FAILED, RUNNING, FINISHING, 
-  FINISHED, KILLED, ALLOCATED_SAVING, LAUNCHED_UNMANAGED_SAVING, RECOVERED,
-  FINAL_SAVING
+  FINISHED, KILLED, ALLOCATED_SAVING, LAUNCHED_UNMANAGED_SAVING, FINAL_SAVING
 }
 }

+ 190 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java

@@ -34,6 +34,10 @@ import java.util.TimerTask;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -48,10 +52,15 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 
 /**
 /**
  * Service to renew application delegation tokens.
  * Service to renew application delegation tokens.
@@ -72,7 +81,8 @@ public class DelegationTokenRenewer extends AbstractService {
   // delegation token canceler thread
   // delegation token canceler thread
   private DelegationTokenCancelThread dtCancelThread =
   private DelegationTokenCancelThread dtCancelThread =
     new DelegationTokenCancelThread();
     new DelegationTokenCancelThread();
-
+  private ThreadPoolExecutor renewerService;
+  
   // managing the list of tokens using Map
   // managing the list of tokens using Map
   // appId=>List<tokens>
   // appId=>List<tokens>
   private Set<DelegationTokenToRenew> delegationTokens = 
   private Set<DelegationTokenToRenew> delegationTokens = 
@@ -84,9 +94,9 @@ public class DelegationTokenRenewer extends AbstractService {
   private long tokenRemovalDelayMs;
   private long tokenRemovalDelayMs;
   
   
   private Thread delayedRemovalThread;
   private Thread delayedRemovalThread;
-  private boolean isServiceStarted = false;
-  private List<DelegationTokenToRenew> pendingTokenForRenewal =
-      new ArrayList<DelegationTokenRenewer.DelegationTokenToRenew>();
+  private ReadWriteLock serviceStateLock = new ReentrantReadWriteLock();
+  private volatile boolean isServiceStarted;
+  private LinkedBlockingQueue<DelegationTokenRenewerEvent> pendingEventQueue;
   
   
   private boolean tokenKeepAliveEnabled;
   private boolean tokenKeepAliveEnabled;
   
   
@@ -102,9 +112,27 @@ public class DelegationTokenRenewer extends AbstractService {
     this.tokenRemovalDelayMs =
     this.tokenRemovalDelayMs =
         conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
         conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
+    renewerService = createNewThreadPoolService(conf);
+    pendingEventQueue = new LinkedBlockingQueue<DelegationTokenRenewerEvent>();
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
 
 
+  protected ThreadPoolExecutor createNewThreadPoolService(Configuration conf) {
+    int nThreads = conf.getInt(
+        YarnConfiguration.RM_DELEGATION_TOKEN_RENEWER_THREAD_COUNT,
+        YarnConfiguration.DEFAULT_RM_DELEGATION_TOKEN_RENEWER_THREAD_COUNT);
+
+    ThreadFactory tf = new ThreadFactoryBuilder()
+        .setNameFormat("DelegationTokenRenewer #%d")
+        .build();
+    ThreadPoolExecutor pool =
+        new ThreadPoolExecutor((5 < nThreads ? 5 : nThreads), nThreads, 3L,
+            TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
+    pool.setThreadFactory(tf);
+    pool.allowCoreThreadTimeOut(true);
+    return pool;
+  }
+
   @Override
   @Override
   protected void serviceStart() throws Exception {
   protected void serviceStart() throws Exception {
     dtCancelThread.start();
     dtCancelThread.start();
@@ -119,21 +147,36 @@ public class DelegationTokenRenewer extends AbstractService {
     RMDelegationTokenIdentifier.Renewer.setSecretManager(
     RMDelegationTokenIdentifier.Renewer.setSecretManager(
         rmContext.getRMDelegationTokenSecretManager(),
         rmContext.getRMDelegationTokenSecretManager(),
         rmContext.getClientRMService().getBindAddress());
         rmContext.getClientRMService().getBindAddress());
-    // Delegation token renewal is delayed until ClientRMService starts. As
-    // it is required to short circuit the token renewal calls.
+    serviceStateLock.writeLock().lock();
     isServiceStarted = true;
     isServiceStarted = true;
-    renewIfServiceIsStarted(pendingTokenForRenewal);
-    pendingTokenForRenewal.clear();
+    serviceStateLock.writeLock().unlock();
+    while(!pendingEventQueue.isEmpty()) {
+      processDelegationTokenRewewerEvent(pendingEventQueue.take());
+    }
     super.serviceStart();
     super.serviceStart();
   }
   }
 
 
+  private void processDelegationTokenRewewerEvent(
+      DelegationTokenRenewerEvent evt) {
+    serviceStateLock.readLock().lock();
+    try {
+      if (isServiceStarted) {
+        renewerService.execute(new DelegationTokenRenewerRunnable(evt));
+      } else {
+        pendingEventQueue.add(evt);
+      }
+    } finally {
+      serviceStateLock.readLock().unlock();
+    }
+  }
+
   @Override
   @Override
   protected void serviceStop() {
   protected void serviceStop() {
     if (renewalTimer != null) {
     if (renewalTimer != null) {
       renewalTimer.cancel();
       renewalTimer.cancel();
     }
     }
     delegationTokens.clear();
     delegationTokens.clear();
-
+    this.renewerService.shutdown();
     dtCancelThread.interrupt();
     dtCancelThread.interrupt();
     try {
     try {
       dtCancelThread.join(1000);
       dtCancelThread.join(1000);
@@ -290,47 +333,50 @@ public class DelegationTokenRenewer extends AbstractService {
    * @throws IOException
    * @throws IOException
    */
    */
   public void addApplication(
   public void addApplication(
-      ApplicationId applicationId, Credentials ts, boolean shouldCancelAtEnd)
+      ApplicationId applicationId, Credentials ts, boolean shouldCancelAtEnd,
+      boolean isApplicationRecovered) {
+    processDelegationTokenRewewerEvent(new DelegationTokenRenewerAppSubmitEvent(
+        applicationId, ts,
+        shouldCancelAtEnd, isApplicationRecovered));
+  }
+
+  private void handleAppSubmitEvent(DelegationTokenRenewerAppSubmitEvent evt)
       throws IOException {
       throws IOException {
+    ApplicationId applicationId = evt.getApplicationId();
+    Credentials ts = evt.getCredentials();
+    boolean shouldCancelAtEnd = evt.shouldCancelAtEnd();
     if (ts == null) {
     if (ts == null) {
-      return; //nothing to add
+      return; // nothing to add
     }
     }
-    
+
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Registering tokens for renewal for:" + 
+      LOG.debug("Registering tokens for renewal for:" +
           " appId = " + applicationId);
           " appId = " + applicationId);
     }
     }
-    
-    Collection <Token<?>> tokens = ts.getAllTokens();
+
+    Collection<Token<?>> tokens = ts.getAllTokens();
     long now = System.currentTimeMillis();
     long now = System.currentTimeMillis();
-    
+
     // find tokens for renewal, but don't add timers until we know
     // find tokens for renewal, but don't add timers until we know
     // all renewable tokens are valid
     // all renewable tokens are valid
     // At RM restart it is safe to assume that all the previously added tokens
     // At RM restart it is safe to assume that all the previously added tokens
     // are valid
     // are valid
     List<DelegationTokenToRenew> tokenList =
     List<DelegationTokenToRenew> tokenList =
         new ArrayList<DelegationTokenRenewer.DelegationTokenToRenew>();
         new ArrayList<DelegationTokenRenewer.DelegationTokenToRenew>();
-    for(Token<?> token : tokens) {
+    for (Token<?> token : tokens) {
       if (token.isManaged()) {
       if (token.isManaged()) {
         tokenList.add(new DelegationTokenToRenew(applicationId,
         tokenList.add(new DelegationTokenToRenew(applicationId,
             token, getConfig(), now, shouldCancelAtEnd));
             token, getConfig(), now, shouldCancelAtEnd));
       }
       }
     }
     }
-    if (!tokenList.isEmpty()){
-      renewIfServiceIsStarted(tokenList);
-    }
-  }
-
-  protected void renewIfServiceIsStarted(List<DelegationTokenToRenew> dtrs)
-      throws IOException {
-    if (isServiceStarted) {
+    if (!tokenList.isEmpty()) {
       // Renewing token and adding it to timer calls are separated purposefully
       // Renewing token and adding it to timer calls are separated purposefully
       // If user provides incorrect token then it should not be added for
       // If user provides incorrect token then it should not be added for
       // renewal.
       // renewal.
-      for (DelegationTokenToRenew dtr : dtrs) {
+      for (DelegationTokenToRenew dtr : tokenList) {
         renewToken(dtr);
         renewToken(dtr);
       }
       }
-      for (DelegationTokenToRenew dtr : dtrs) {
+      for (DelegationTokenToRenew dtr : tokenList) {
         addTokenToList(dtr);
         addTokenToList(dtr);
         setTimerForTokenRenewal(dtr);
         setTimerForTokenRenewal(dtr);
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
@@ -338,11 +384,9 @@ public class DelegationTokenRenewer extends AbstractService {
               + dtr.token.getService() + " for appId = " + dtr.applicationId);
               + dtr.token.getService() + " for appId = " + dtr.applicationId);
         }
         }
       }
       }
-    } else {
-      pendingTokenForRenewal.addAll(dtrs);
     }
     }
   }
   }
-  
+
   /**
   /**
    * Task - to renew a token
    * Task - to renew a token
    *
    *
@@ -449,14 +493,20 @@ public class DelegationTokenRenewer extends AbstractService {
    * @param applicationId completed application
    * @param applicationId completed application
    */
    */
   public void applicationFinished(ApplicationId applicationId) {
   public void applicationFinished(ApplicationId applicationId) {
+    processDelegationTokenRewewerEvent(new DelegationTokenRenewerEvent(
+        applicationId,
+        DelegationTokenRenewerEventType.FINISH_APPLICATION));
+  }
+
+  private void handleAppFinishEvent(DelegationTokenRenewerEvent evt) {
     if (!tokenKeepAliveEnabled) {
     if (!tokenKeepAliveEnabled) {
-      removeApplicationFromRenewal(applicationId);
+      removeApplicationFromRenewal(evt.getApplicationId());
     } else {
     } else {
-      delayedRemovalMap.put(applicationId, System.currentTimeMillis()
+      delayedRemovalMap.put(evt.getApplicationId(), System.currentTimeMillis()
           + tokenRemovalDelayMs);
           + tokenRemovalDelayMs);
     }
     }
   }
   }
-
+  
   /**
   /**
    * Add a list of applications to the keep alive list. If an appId already
    * Add a list of applications to the keep alive list. If an appId already
    * exists, update it's keep-alive time.
    * exists, update it's keep-alive time.
@@ -546,4 +596,111 @@ public class DelegationTokenRenewer extends AbstractService {
   public void setRMContext(RMContext rmContext) {
   public void setRMContext(RMContext rmContext) {
     this.rmContext = rmContext;
     this.rmContext = rmContext;
   }
   }
+  
+  /*
+   * This will run as a separate thread and will process individual events. It
+   * is done in this way to make sure that the token renewal as a part of
+   * application submission and token removal as a part of application finish
+   * is asynchronous in nature.
+   */
+  private final class DelegationTokenRenewerRunnable
+      implements Runnable {
+
+    private DelegationTokenRenewerEvent evt;
+    
+    public DelegationTokenRenewerRunnable(DelegationTokenRenewerEvent evt) {
+      this.evt = evt;
+    }
+    
+    @Override
+    public void run() {
+      if (evt instanceof DelegationTokenRenewerAppSubmitEvent) {
+        DelegationTokenRenewerAppSubmitEvent appSubmitEvt =
+            (DelegationTokenRenewerAppSubmitEvent) evt;
+        handleDTRenewerAppSubmitEvent(appSubmitEvt);
+      } else if (evt.getType().equals(
+          DelegationTokenRenewerEventType.FINISH_APPLICATION)) {
+        DelegationTokenRenewer.this.handleAppFinishEvent(evt);
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    private void handleDTRenewerAppSubmitEvent(
+        DelegationTokenRenewerAppSubmitEvent event) {
+      /*
+       * For applications submitted with delegation tokens we are not submitting
+       * the application to scheduler from RMAppManager. Instead we are doing
+       * it from here. The primary goal is to make token renewal as a part of
+       * application submission asynchronous so that client thread is not
+       * blocked during app submission.
+       */
+      try {
+        // Setup tokens for renewal
+        DelegationTokenRenewer.this.handleAppSubmitEvent(event);
+        rmContext.getDispatcher().getEventHandler()
+            .handle(new RMAppEvent(event.getApplicationId(),
+                event.isApplicationRecovered() ? RMAppEventType.RECOVER
+                    : RMAppEventType.START));
+      } catch (Throwable t) {
+        LOG.warn(
+            "Unable to add the application to the delegation token renewer.",
+            t);
+        // Sending APP_REJECTED is fine, since we assume that the
+        // RMApp is in NEW state and thus we havne't yet informed the
+        // Scheduler about the existence of the application
+        rmContext.getDispatcher().getEventHandler().handle(
+            new RMAppRejectedEvent(event.getApplicationId(), t.getMessage()));
+      }
+    }
+  }
+  
+  class DelegationTokenRenewerAppSubmitEvent extends
+      DelegationTokenRenewerEvent {
+
+    private Credentials credentials;
+    private boolean shouldCancelAtEnd;
+    private boolean isAppRecovered;
+
+    public DelegationTokenRenewerAppSubmitEvent(ApplicationId appId,
+        Credentials credentails, boolean shouldCancelAtEnd,
+        boolean isApplicationRecovered) {
+      super(appId, DelegationTokenRenewerEventType.VERIFY_AND_START_APPLICATION);
+      this.credentials = credentails;
+      this.shouldCancelAtEnd = shouldCancelAtEnd;
+      this.isAppRecovered = isApplicationRecovered;
+    }
+
+    public Credentials getCredentials() {
+      return credentials;
+    }
+
+    public boolean shouldCancelAtEnd() {
+      return shouldCancelAtEnd;
+    }
+
+    public boolean isApplicationRecovered() {
+      return isAppRecovered;
+    }
+  }
+  
+  enum DelegationTokenRenewerEventType {
+    VERIFY_AND_START_APPLICATION,
+    FINISH_APPLICATION
+  }
+  
+  class DelegationTokenRenewerEvent extends
+      AbstractEvent<DelegationTokenRenewerEventType> {
+
+    private ApplicationId appId;
+
+    public DelegationTokenRenewerEvent(ApplicationId appId,
+        DelegationTokenRenewerEventType type) {
+      super(type);
+      this.appId = appId;
+    }
+
+    public ApplicationId getApplicationId() {
+      return appId;
+    }
+  }
 }
 }

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java

@@ -249,7 +249,8 @@ class CapacitySchedulerPage extends RmView {
         _("$(function() {",
         _("$(function() {",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
-          "    data.inst.open_node('#pq', true);",
+          "    var callback = { call:reopenQueryNodes }",
+          "    data.inst.open_node('#pq', callback);",
           "   }).",
           "   }).",
           "    jstree({",
           "    jstree({",
           "    core: { animation: 188, html_titles: true },",
           "    core: { animation: 188, html_titles: true },",
@@ -265,7 +266,8 @@ class CapacitySchedulerPage extends RmView {
           "    $('#apps').dataTable().fnFilter(q, 3, true);",
           "    $('#apps').dataTable().fnFilter(q, 3, true);",
           "  });",
           "  });",
           "  $('#cs').show();",
           "  $('#cs').show();",
-          "});")._();
+          "});")._().
+      _(SchedulerPageUtil.QueueBlockUtil.class);
   }
   }
 
 
   @Override protected Class<? extends SubView> content() {
   @Override protected Class<? extends SubView> content() {

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java

@@ -201,7 +201,8 @@ public class FairSchedulerPage extends RmView {
         _("$(function() {",
         _("$(function() {",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
-          "    data.inst.open_node('#pq', true);",
+          "    var callback = { call:reopenQueryNodes }",
+          "    data.inst.open_node('#pq', callback);",
           "   }).",
           "   }).",
           "    jstree({",
           "    jstree({",
           "    core: { animation: 188, html_titles: true },",
           "    core: { animation: 188, html_titles: true },",
@@ -217,7 +218,8 @@ public class FairSchedulerPage extends RmView {
           "    $('#apps').dataTable().fnFilter(q, 3, true);",
           "    $('#apps').dataTable().fnFilter(q, 3, true);",
           "  });",
           "  });",
           "  $('#cs').show();",
           "  $('#cs').show();",
-          "});")._();
+          "});")._().
+        _(SchedulerPageUtil.QueueBlockUtil.class);
   }
   }
   
   
   @Override protected Class<? extends SubView> content() {
   @Override protected Class<? extends SubView> content() {

+ 177 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/SchedulerPageUtil.java

@@ -0,0 +1,177 @@
+/**
+* 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.resourcemanager.webapp;
+
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+public class SchedulerPageUtil {
+
+  static class QueueBlockUtil extends HtmlBlock {
+
+    private void reopenQueue(Block html) {
+      html.
+          script().$type("text/javascript").
+          _("function reopenQueryNodes() {",
+            "  var currentParam = window.location.href.split('?');",
+            "  var tmpCurrentParam = currentParam;",
+            "  var queryQueuesString = '';",
+            "  if (tmpCurrentParam.length > 1) {",
+            "    // openQueues=q1#q2&param1=value1&param2=value2",
+            "    tmpCurrentParam = tmpCurrentParam[1];",
+            "    if (tmpCurrentParam.indexOf('openQueues=') != -1 ) {",
+            "      tmpCurrentParam = tmpCurrentParam.split('openQueues=')[1].split('&')[0];",
+            "      queryQueuesString = tmpCurrentParam;",
+            "    }",
+            "  }",
+            "  if (queryQueuesString != '') {",
+            "    queueArray = queryQueuesString.split('#');",
+            "    $('#cs .q').each(function() {",
+            "      var name = $(this).html();",
+            "      if (name != 'root' && $.inArray(name, queueArray) != -1) {",
+            "        $(this).closest('li').removeClass('jstree-closed').addClass('jstree-open'); ",
+            "      }",
+            "    });",
+            "  }",
+            "  $('#cs').bind( {",
+            "                  'open_node.jstree' :function(e, data) { storeExpandedQueue(e, data); },",
+            "                  'close_node.jstree':function(e, data) { storeExpandedQueue(e, data); }",
+            "  });",
+            "}")._();
+    }
+
+    private void storeExpandedQueue (Block html) {
+      html.
+          script().$type("text/javascript").
+          _("function storeExpandedQueue(e, data) {",
+            "  var OPEN_QUEUES = 'openQueues';",
+            "  var ACTION_OPEN = 'open';",
+            "  var ACTION_CLOSED = 'closed';",
+            "  var $li = $(data.args[0]);",
+            "  var action = ACTION_CLOSED;  //closed or open",
+            "  var queueName = ''",
+            "  if ($li.hasClass('jstree-open')) {",
+            "      action=ACTION_OPEN;",
+            "  }",
+            "  queueName = $li.find('.q').html();",
+            "  // http://localhost:8088/cluster/scheduler?openQueues=q1#q2&param1=value1&param2=value2 ",
+            "  //   ==> [http://localhost:8088/cluster/scheduler , openQueues=q1#q2&param1=value1&param2=value2]",
+            "  var currentParam = window.location.href.split('?');",
+            "  var tmpCurrentParam = currentParam;",
+            "  var queryString = '';",
+            "  if (tmpCurrentParam.length > 1) {",
+            "    // openQueues=q1#q2&param1=value1&param2=value2",
+            "    tmpCurrentParam = tmpCurrentParam[1];",
+            "    currentParam = tmpCurrentParam;",
+            "    tmpCurrentParam = tmpCurrentParam.split('&');",
+            "    var len = tmpCurrentParam.length;",
+            "    var paramExist = false;",
+            "    if (len > 1) {    // Currently no query param are present but in future if any are added for that handling it now",
+            "      queryString = '';",
+            "      for (var i = 0 ; i < len ; i++) {  // searching for param openQueues",
+            "        if (tmpCurrentParam[i].substr(0,11) == OPEN_QUEUES + '=') {",
+            "          if (action == ACTION_OPEN) {",
+            "            tmpCurrentParam[i] = addQueueName(tmpCurrentParam[i],queueName);",
+            "          }",
+            "          else if (action == ACTION_CLOSED) {",
+            "            tmpCurrentParam[i] = removeQueueName(tmpCurrentParam[i] , queueName);",
+            "          }",
+            "          paramExist = true;",
+            "        }",
+            "        if (i > 0) {",
+            "          queryString += '&';",
+            "        }",
+            "        queryString += tmpCurrentParam[i];",
+            "      }",
+            "      // If in existing query string OPEN_QUEUES param is not present",
+            "      if (action == ACTION_OPEN && !paramExist) {",
+            "        queryString = currentParam + '&' + OPEN_QUEUES + '=' + queueName;",
+            "      }",
+            "    } ",
+            "    // Only one param is present in current query string",
+            "    else {",
+            "      tmpCurrentParam=tmpCurrentParam[0];",
+            "      // checking if the only param present in query string is OPEN_QUEUES or not and making queryString accordingly",
+            "      if (tmpCurrentParam.substr(0,11) == OPEN_QUEUES + '=') {",
+            "        if (action == ACTION_OPEN) {",
+            "          queryString = addQueueName(tmpCurrentParam,queueName);",
+            "        }",
+            "        else if (action == ACTION_CLOSED) {",
+            "          queryString = removeQueueName(tmpCurrentParam , queueName);",
+            "        }",
+            "      }",
+            "      else {",
+            "        if (action == ACTION_OPEN) {",
+            "          queryString = tmpCurrentParam + '&' + OPEN_QUEUES + '=' + queueName;",
+            "        }",
+            "      }",
+            "    }",
+            "  } else {",
+            "    if (action == ACTION_OPEN) {",
+            "      tmpCurrentParam = '';",
+            "      currentParam = tmpCurrentParam;",
+            "      queryString = OPEN_QUEUES+'='+queueName;",
+            "    }",
+            "  }",
+            "  if (queryString != '') {",
+            "    queryString = '?' + queryString;",
+            "  }",
+            "  var url = window.location.protocol + '//' + window.location.host + window.location.pathname + queryString;",
+            "  window.history.pushState( { path : url }, '', url);",
+            "};",
+            "",
+            "function removeQueueName(queryString, queueName) {",
+            "  var index = queryString.indexOf(queueName);",
+            "  // Finding if queue is present in query param then only remove it",
+            "  if (index != -1) {",
+            "    // removing openQueues=",
+            "    var tmp = queryString.substr(11, queryString.length);",
+            "    tmp = tmp.split('#');",
+            "    var len = tmp.length;",
+            "    var newQueryString = '';",
+            "    for (var i = 0 ; i < len ; i++) {",
+            "      if (tmp[i] != queueName) {",
+            "        if (newQueryString != '') {",
+            "          newQueryString += '#';",
+            "        }",
+            "        newQueryString += tmp[i];",
+            "      }",
+            "    }",
+            "    queryString = newQueryString;",
+            "    if (newQueryString != '') {",
+            "      queryString = 'openQueues=' + newQueryString;",
+            "    }",
+            "  }",
+            "  return queryString;",
+            "}",
+            "",
+            "function addQueueName(queryString, queueName) {",
+            "  queueArray = queryString.split('#');",
+            "  if ($.inArray(queueArray, queueName) == -1) {",
+            "    queryString = queryString + '#' + queueName;",
+            "  }",
+            "  return queryString;",
+            "}")._();
+    }
+
+    @Override protected void render(Block html) {
+      reopenQueue(html);
+      storeExpandedQueue(html);
+    }
+  }
+}

+ 10 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java

@@ -18,6 +18,7 @@
 
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -31,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
@@ -98,21 +100,27 @@ public class MockNM {
   }
   }
 
 
   public RegisterNodeManagerResponse registerNode() throws Exception {
   public RegisterNodeManagerResponse registerNode() throws Exception {
+    return registerNode(null);
+  }
+
+  public RegisterNodeManagerResponse registerNode(
+      List<ContainerStatus> containerStatus) throws Exception{
     RegisterNodeManagerRequest req = Records.newRecord(
     RegisterNodeManagerRequest req = Records.newRecord(
         RegisterNodeManagerRequest.class);
         RegisterNodeManagerRequest.class);
     req.setNodeId(nodeId);
     req.setNodeId(nodeId);
     req.setHttpPort(httpPort);
     req.setHttpPort(httpPort);
     Resource resource = BuilderUtils.newResource(memory, vCores);
     Resource resource = BuilderUtils.newResource(memory, vCores);
     req.setResource(resource);
     req.setResource(resource);
+    req.setContainerStatuses(containerStatus);
     req.setNMVersion(version);
     req.setNMVersion(version);
     RegisterNodeManagerResponse registrationResponse =
     RegisterNodeManagerResponse registrationResponse =
         resourceTracker.registerNodeManager(req);
         resourceTracker.registerNodeManager(req);
     this.currentContainerTokenMasterKey =
     this.currentContainerTokenMasterKey =
         registrationResponse.getContainerTokenMasterKey();
         registrationResponse.getContainerTokenMasterKey();
     this.currentNMTokenMasterKey = registrationResponse.getNMTokenMasterKey();
     this.currentNMTokenMasterKey = registrationResponse.getNMTokenMasterKey();
-    return registrationResponse;
+    return registrationResponse;    
   }
   }
-
+  
   public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception {
   public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception {
     return nodeHeartbeat(new HashMap<ApplicationId, List<ContainerStatus>>(),
     return nodeHeartbeat(new HashMap<ApplicationId, List<ContainerStatus>>(),
         isHealthy, ++responseId);
         isHealthy, ++responseId);

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -421,6 +421,10 @@ public class MockRM extends ResourceManager {
     return this.clientToAMSecretManager;
     return this.clientToAMSecretManager;
   }
   }
 
 
+  public RMAppManager getRMAppManager() {
+    return this.rmAppManager;
+  }
+
   @Override
   @Override
   protected void startWepApp() {
   protected void startWepApp() {
     // override to disable webapp
     // override to disable webapp

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

@@ -172,7 +172,7 @@ public class TestAppManager{
         ApplicationSubmissionContext submissionContext, String user)
         ApplicationSubmissionContext submissionContext, String user)
             throws YarnException {
             throws YarnException {
       super.submitApplication(submissionContext, System.currentTimeMillis(),
       super.submitApplication(submissionContext, System.currentTimeMillis(),
-          false, user);
+          user, false, null);
     }
     }
   }
   }
 
 

+ 253 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -18,6 +18,11 @@
 
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
@@ -29,6 +34,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -57,10 +63,14 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -83,6 +93,7 @@ import org.apache.log4j.Logger;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+import org.mortbay.log.Log;
 
 
 public class TestRMRestart {
 public class TestRMRestart {
 
 
@@ -104,6 +115,7 @@ public class TestRMRestart {
     Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
     Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
   }
   }
 
 
+  @SuppressWarnings("rawtypes")
   @Test (timeout=180000)
   @Test (timeout=180000)
   public void testRMRestart() throws Exception {
   public void testRMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
@@ -252,11 +264,14 @@ public class TestRMRestart {
         .getApplicationId());
         .getApplicationId());
     
     
     // verify state machine kicked into expected states
     // verify state machine kicked into expected states
-    rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
+    rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.RUNNING);
     rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.ACCEPTED);
     
     
-    // verify new attempts created
-    Assert.assertEquals(2, loadedApp1.getAppAttempts().size());
+    // verify attempts for apps
+    // The app for which AM was started will wait for previous am
+    // container finish event to arrive. However for an application for which
+    // no am container was running will start new application attempt.
+    Assert.assertEquals(1, loadedApp1.getAppAttempts().size());
     Assert.assertEquals(1, loadedApp2.getAppAttempts().size());
     Assert.assertEquals(1, loadedApp2.getAppAttempts().size());
     
     
     // verify old AM is not accepted
     // verify old AM is not accepted
@@ -274,8 +289,20 @@ public class TestRMRestart {
     Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction());
     Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction());
     
     
     // new NM to represent NM re-register
     // new NM to represent NM re-register
-    nm1 = rm2.registerNode("127.0.0.1:1234", 15120);
-    nm2 = rm2.registerNode("127.0.0.2:5678", 15120);
+    nm1 = new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService());
+    nm2 = new MockNM("127.0.0.2:5678", 15120, rm2.getResourceTrackerService());
+
+    List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(BuilderUtils.newContainerId(loadedApp1
+            .getCurrentAppAttempt().getAppAttemptId(), 1),
+            ContainerState.COMPLETE, "Killed AM container", 143);
+    containerStatuses.add(containerStatus);
+    nm1.registerNode(containerStatuses);
+    nm2.registerNode();
+    
+    rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
+    Assert.assertEquals(2, loadedApp1.getAppAttempts().size());    
 
 
     // verify no more reboot response sent
     // verify no more reboot response sent
     hbResponse = nm1.nodeHeartbeat(true);
     hbResponse = nm1.nodeHeartbeat(true);
@@ -398,6 +425,157 @@ public class TestRMRestart {
       .getAppAttempts().get(am0.getApplicationAttemptId()).getAppAttemptState());
       .getAppAttempts().get(am0.getApplicationAttemptId()).getAppAttemptState());
   }
   }
 
 
+  @Test
+  public void testRMRestartWaitForPreviousAMToFinish() throws Exception {
+    // testing 3 cases
+    // After RM restarts
+    // 1) New application attempt is not started until previous AM container
+    // finish event is reported back to RM as a part of nm registration.
+    // 2) If previous AM container finish event is never reported back (i.e.
+    // node manager on which this AM container was running also went down) in
+    // that case AMLivenessMonitor should time out previous attempt and start
+    // new attempt.
+    // 3) If all the stored attempts had finished then new attempt should
+    // be started immediately.
+    YarnConfiguration conf = new YarnConfiguration(this.conf);
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 40);
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    RMState rmState = memStore.getState();
+    Map<ApplicationId, ApplicationState> rmAppState =
+        rmState.getApplicationState();
+    
+    // start RM
+    final MockRM rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234" , 16382, rm1.getResourceTrackerService());
+    nm1.registerNode();
+     
+    // submitting app
+    RMApp app1 = rm1.submitApp(200);
+    rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
+    MockAM am1 = launchAM(app1, rm1, nm1);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
+    // Fail first AM.
+    am1.waitForState(RMAppAttemptState.FAILED);
+    
+    // launch another AM.
+    MockAM am2 = launchAM(app1, rm1, nm1);
+    
+    Assert.assertEquals(1, rmAppState.size());
+    Assert.assertEquals(app1.getState(), RMAppState.RUNNING);
+    Assert.assertEquals(app1.getAppAttempts()
+        .get(app1.getCurrentAppAttempt().getAppAttemptId())
+        .getAppAttemptState(), RMAppAttemptState.RUNNING);
+
+    //  start new RM.
+    MockRM rm2 = null;
+    rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    NodeHeartbeatResponse res = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(NodeAction.RESYNC, res.getNodeAction());
+    
+    RMApp rmApp = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
+    // application should be in running state
+    rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    
+    Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
+    // new attempt should not be started
+    Assert.assertEquals(2, rmApp.getAppAttempts().size());
+    // am1 attempt should be in FAILED state where as am2 attempt should be in
+    // LAUNCHED state
+    Assert.assertEquals(RMAppAttemptState.FAILED,
+        rmApp.getAppAttempts().get(am1.getApplicationAttemptId())
+            .getAppAttemptState());
+    Assert.assertEquals(RMAppAttemptState.LAUNCHED,
+        rmApp.getAppAttempts().get(am2.getApplicationAttemptId())
+            .getAppAttemptState());
+    
+    List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(
+            BuilderUtils.newContainerId(am2.getApplicationAttemptId(), 1),
+            ContainerState.COMPLETE, "Killed AM container", 143);
+    containerStatuses.add(containerStatus);
+    nm1.registerNode(containerStatuses);
+    rm2.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.FAILED);
+    launchAM(rmApp, rm2, nm1);
+    Assert.assertEquals(3, rmApp.getAppAttempts().size());
+    rm2.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(),
+        RMAppAttemptState.RUNNING);
+    // Now restart RM ...
+    // Setting AMLivelinessMonitor interval to be 10 Secs. 
+    conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 10000);
+    MockRM rm3 = null;
+    rm3 = new MockRM(conf, memStore);
+    rm3.start();
+    
+    // Wait for RM to process all the events as a part of rm recovery.
+    nm1.setResourceTrackerService(rm3.getResourceTrackerService());
+    
+    rmApp = rm3.getRMContext().getRMApps().get(app1.getApplicationId());
+    // application should be in running state
+    rm3.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    Assert.assertEquals(rmApp.getState(), RMAppState.RUNNING);
+    // new attempt should not be started
+    Assert.assertEquals(3, rmApp.getAppAttempts().size());
+    // am1 and am2 attempts should be in FAILED state where as am3 should be
+    // in LAUNCHED state
+    Assert.assertEquals(RMAppAttemptState.FAILED,
+        rmApp.getAppAttempts().get(am1.getApplicationAttemptId())
+            .getAppAttemptState());
+    Assert.assertEquals(RMAppAttemptState.FAILED,
+        rmApp.getAppAttempts().get(am2.getApplicationAttemptId())
+            .getAppAttemptState());
+    ApplicationAttemptId latestAppAttemptId =
+        rmApp.getCurrentAppAttempt().getAppAttemptId();
+    Assert.assertEquals(RMAppAttemptState.LAUNCHED,rmApp.getAppAttempts()
+        .get(latestAppAttemptId).getAppAttemptState());
+    
+    rm3.waitForState(latestAppAttemptId, RMAppAttemptState.FAILED);
+    rm3.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
+    Assert.assertEquals(4, rmApp.getAppAttempts().size());
+    Assert.assertEquals(RMAppAttemptState.FAILED,
+        rmApp.getAppAttempts().get(latestAppAttemptId).getAppAttemptState());
+    
+    latestAppAttemptId = rmApp.getCurrentAppAttempt().getAppAttemptId();
+    
+    // The 4th attempt has started but is not yet saved into RMStateStore
+    // It will be saved only when we launch AM.
+
+    // submitting app but not starting AM for it.
+    RMApp app2 = rm3.submitApp(200);
+    rm3.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED);
+    Assert.assertEquals(1, app2.getAppAttempts().size());
+    Assert.assertEquals(0,
+        memStore.getState().getApplicationState().get(app2.getApplicationId())
+            .getAttemptCount());
+
+    MockRM rm4 = null;
+    rm4 = new MockRM(conf, memStore);
+    rm4.start();
+    
+    rmApp = rm4.getRMContext().getRMApps().get(app1.getApplicationId());
+    rm4.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
+    Assert.assertEquals(4, rmApp.getAppAttempts().size());
+    Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState());
+    Assert.assertEquals(RMAppAttemptState.SCHEDULED, rmApp.getAppAttempts()
+        .get(latestAppAttemptId).getAppAttemptState());
+    
+    // The initial application for which an AM was not started should be in
+    // ACCEPTED state with one application attempt started.
+    app2 = rm4.getRMContext().getRMApps().get(app2.getApplicationId());
+    rm4.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED);
+    Assert.assertEquals(RMAppState.ACCEPTED, app2.getState());
+    Assert.assertEquals(1, app2.getAppAttempts().size());
+    Assert.assertEquals(RMAppAttemptState.SCHEDULED, app2
+        .getCurrentAppAttempt().getAppAttemptState());
+
+  }
+
   @Test
   @Test
   public void testRMRestartFailedApp() throws Exception {
   public void testRMRestartFailedApp() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
@@ -577,7 +755,14 @@ public class TestRMRestart {
     rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.KILLED);
     rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.KILLED);
 
 
     // restart rm
     // restart rm
-    MockRM rm2 = new MockRM(conf, memStore);
+
+    MockRM rm2 = new MockRM(conf, memStore) {
+      @Override
+      protected RMAppManager createRMAppManager() {
+        return spy(super.createRMAppManager());
+      }
+    };
+
     rm2.start();
     rm2.start();
 
 
     GetApplicationsRequest request1 =
     GetApplicationsRequest request1 =
@@ -620,6 +805,10 @@ public class TestRMRestart {
         rm2.getClientRMService().getApplications(request2);
         rm2.getClientRMService().getApplications(request2);
     List<ApplicationReport> appList2 = response2.getApplicationList();
     List<ApplicationReport> appList2 = response2.getApplicationList();
     Assert.assertTrue(3 == appList2.size());
     Assert.assertTrue(3 == appList2.size());
+
+    // check application summary is logged for the completed apps after RM restart.
+    verify(rm2.getRMAppManager(), times(3)).logApplicationSummary(
+      isA(ApplicationId.class));
   }
   }
 
 
   private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
   private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
@@ -720,6 +909,8 @@ public class TestRMRestart {
     Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1), 
     Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1), 
                         attemptState.getMasterContainer().getId());
                         attemptState.getMasterContainer().getId());
 
 
+    // Setting AMLivelinessMonitor interval to be 10 Secs. 
+    conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 10000);
     // start new RM   
     // start new RM   
     MockRM rm2 = new MockRM(conf, memStore);
     MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
     rm2.start();
@@ -818,6 +1009,10 @@ public class TestRMRestart {
     MockRM rm2 = new TestSecurityMockRM(conf, memStore);
     MockRM rm2 = new TestSecurityMockRM(conf, memStore);
     rm2.start();
     rm2.start();
 
 
+    // Need to wait for a while as now token renewal happens on another thread
+    // and is asynchronous in nature.
+    waitForTokensToBeRenewed(rm2);
+
     // verify tokens are properly populated back to rm2 DelegationTokenRenewer
     // verify tokens are properly populated back to rm2 DelegationTokenRenewer
     Assert.assertEquals(tokenSet, rm2.getRMContext()
     Assert.assertEquals(tokenSet, rm2.getRMContext()
       .getDelegationTokenRenewer().getDelegationTokens());
       .getDelegationTokenRenewer().getDelegationTokens());
@@ -827,6 +1022,21 @@ public class TestRMRestart {
     rm2.stop();
     rm2.stop();
   }
   }
 
 
+  private void waitForTokensToBeRenewed(MockRM rm2) throws Exception {
+    int waitCnt = 20;
+    boolean atleastOneAppInNEWState = true;
+    while (waitCnt-- > 0 && atleastOneAppInNEWState) {
+      atleastOneAppInNEWState = false;
+      for (RMApp rmApp : rm2.getRMContext().getRMApps().values()) {
+        if (rmApp.getState() == RMAppState.NEW) {
+          Thread.sleep(1000);
+          atleastOneAppInNEWState = true;
+          break;
+        }
+      }
+    }
+  }
+
   @Test
   @Test
   public void testAppAttemptTokensRestoredOnRMRestart() throws Exception {
   public void testAppAttemptTokensRestoredOnRMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
@@ -920,7 +1130,6 @@ public class TestRMRestart {
   @Test
   @Test
   public void testRMDelegationTokenRestoredOnRMRestart() throws Exception {
   public void testRMDelegationTokenRestoredOnRMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    
     conf.set(
     conf.set(
         CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
         CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
         "kerberos");
         "kerberos");
@@ -1063,6 +1272,43 @@ public class TestRMRestart {
     rm2.stop();
     rm2.stop();
   }
   }
 
 
+  // This is to test submit an application to the new RM with the old delegation
+  // token got from previous RM.
+  @Test
+  public void testAppSubmissionWithOldDelegationTokenAfterRMRestart()
+      throws Exception {
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "kerberos");
+    conf.set(YarnConfiguration.RM_ADDRESS, "localhost:8032");
+    UserGroupInformation.setConfiguration(conf);
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+
+    MockRM rm1 = new TestSecurityMockRM(conf, memStore);
+    rm1.start();
+
+    GetDelegationTokenRequest request1 =
+        GetDelegationTokenRequest.newInstance("renewer1");
+    UserGroupInformation.getCurrentUser().setAuthenticationMethod(
+        AuthMethod.KERBEROS);
+    GetDelegationTokenResponse response1 =
+        rm1.getClientRMService().getDelegationToken(request1);
+    Token<RMDelegationTokenIdentifier> token1 =
+        ConverterUtils.convertFromYarn(response1.getRMDelegationToken(), rmAddr);
+
+    // start new RM
+    MockRM rm2 = new TestSecurityMockRM(conf, memStore);
+    rm2.start();
+
+    // submit an app with the old delegation token got from previous RM.
+    Credentials ts = new Credentials();
+    ts.addToken(token1.getService(), token1);
+    RMApp app = rm2.submitApp(200, "name", "user",
+        new HashMap<ApplicationAccessType, String>(), false, "default", 1, ts);
+    rm2.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
+  }
+
   @Test
   @Test
   public void testRMStateStoreDispatcherDrainedOnRMStop() throws Exception {
   public void testRMStateStoreDispatcherDrainedOnRMStop() throws Exception {
     MemoryRMStateStore memStore = new MemoryRMStateStore() {
     MemoryRMStateStore memStore = new MemoryRMStateStore() {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -372,10 +372,10 @@ public class TestRMAppAttemptTransitions {
   }
   }
   
   
   /**
   /**
-   * {@link RMAppAttemptState#RECOVERED}
+   * {@link RMAppAttemptState#LAUNCHED}
    */
    */
   private void testAppAttemptRecoveredState() {
   private void testAppAttemptRecoveredState() {
-    assertEquals(RMAppAttemptState.RECOVERED, 
+    assertEquals(RMAppAttemptState.LAUNCHED, 
         applicationAttempt.getAppAttemptState());
         applicationAttempt.getAppAttemptState());
   }
   }
 
 

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -141,6 +141,8 @@ public class TestFifoScheduler {
 
 
     FifoScheduler schedular = new FifoScheduler();
     FifoScheduler schedular = new FifoScheduler();
     schedular.reinitialize(new Configuration(), rmContext);
     schedular.reinitialize(new Configuration(), rmContext);
+    QueueMetrics metrics = schedular.getRootQueueMetrics();
+    int beforeAppsSubmitted = metrics.getAppsSubmitted();
 
 
     ApplicationId appId = BuilderUtils.newApplicationId(200, 1);
     ApplicationId appId = BuilderUtils.newApplicationId(200, 1);
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
@@ -155,8 +157,8 @@ public class TestFifoScheduler {
     event = new AppAddedSchedulerEvent(appAttemptId, "queue", "user");
     event = new AppAddedSchedulerEvent(appAttemptId, "queue", "user");
     schedular.handle(event);
     schedular.handle(event);
 
 
-    QueueMetrics metrics = schedular.getRootQueueMetrics();
-    Assert.assertEquals(1, metrics.getAppsSubmitted());
+    int afterAppsSubmitted = metrics.getAppsSubmitted();
+    Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted);
   }
   }
 
 
   @Test(timeout=2000)
   @Test(timeout=2000)

+ 212 - 84
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java

@@ -31,13 +31,24 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import junit.framework.Assert;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -46,16 +57,29 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
 import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -66,14 +90,18 @@ import org.mockito.stubbing.Answer;
 
 
 /**
 /**
  * unit test - 
  * unit test - 
- * tests addition/deletion/cancelation of renewals of delegation tokens
+ * tests addition/deletion/cancellation of renewals of delegation tokens
  *
  *
  */
  */
+@SuppressWarnings("rawtypes")
 public class TestDelegationTokenRenewer {
 public class TestDelegationTokenRenewer {
   private static final Log LOG = 
   private static final Log LOG = 
       LogFactory.getLog(TestDelegationTokenRenewer.class);
       LogFactory.getLog(TestDelegationTokenRenewer.class);
   private static final Text KIND = new Text("TestDelegationTokenRenewer.Token");
   private static final Text KIND = new Text("TestDelegationTokenRenewer.Token");
   
   
+  private static BlockingQueue<Event> eventQueue;
+  private static volatile AtomicInteger counter;
+  private static AsyncDispatcher dispatcher;
   public static class Renewer extends TokenRenewer {
   public static class Renewer extends TokenRenewer {
     private static int counter = 0;
     private static int counter = 0;
     private static Token<?> lastRenewed = null;
     private static Token<?> lastRenewed = null;
@@ -143,11 +171,20 @@ public class TestDelegationTokenRenewer {
 
 
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
+    counter = new AtomicInteger(0);
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    eventQueue = new LinkedBlockingQueue<Event>();
+    dispatcher = new AsyncDispatcher(eventQueue);
     Renewer.reset();
     Renewer.reset();
-    delegationTokenRenewer = new DelegationTokenRenewer();
+    delegationTokenRenewer = createNewDelegationTokenRenewer(conf, counter);
     delegationTokenRenewer.init(conf);
     delegationTokenRenewer.init(conf);
     RMContext mockContext = mock(RMContext.class);
     RMContext mockContext = mock(RMContext.class);
     ClientRMService mockClientRMService = mock(ClientRMService.class);
     ClientRMService mockClientRMService = mock(ClientRMService.class);
+    when(mockContext.getDelegationTokenRenewer()).thenReturn(
+        delegationTokenRenewer);
+    when(mockContext.getDispatcher()).thenReturn(dispatcher);
     when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
     when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
     InetSocketAddress sockAddr =
     InetSocketAddress sockAddr =
         InetSocketAddress.createUnresolved("localhost", 1234);
         InetSocketAddress.createUnresolved("localhost", 1234);
@@ -285,7 +322,7 @@ public class TestDelegationTokenRenewer {
    * @throws IOException
    * @throws IOException
    * @throws URISyntaxException
    * @throws URISyntaxException
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testDTRenewal () throws Exception {
   public void testDTRenewal () throws Exception {
     MyFS dfs = (MyFS)FileSystem.get(conf);
     MyFS dfs = (MyFS)FileSystem.get(conf);
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
@@ -316,8 +353,9 @@ public class TestDelegationTokenRenewer {
     // register the tokens for renewal
     // register the tokens for renewal
     ApplicationId applicationId_0 = 
     ApplicationId applicationId_0 = 
         BuilderUtils.newApplicationId(0, 0);
         BuilderUtils.newApplicationId(0, 0);
-    delegationTokenRenewer.addApplication(applicationId_0, ts, true);
-    
+    delegationTokenRenewer.addApplication(applicationId_0, ts, true, false);
+    waitForEventsToGetProcessed(delegationTokenRenewer);
+
     // first 3 initial renewals + 1 real
     // first 3 initial renewals + 1 real
     int numberOfExpectedRenewals = 3+1; 
     int numberOfExpectedRenewals = 3+1; 
     
     
@@ -355,9 +393,10 @@ public class TestDelegationTokenRenewer {
     
     
 
 
     ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
     ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
-    delegationTokenRenewer.addApplication(applicationId_1, ts, true);
+    delegationTokenRenewer.addApplication(applicationId_1, ts, true, false);
+    waitForEventsToGetProcessed(delegationTokenRenewer);
     delegationTokenRenewer.applicationFinished(applicationId_1);
     delegationTokenRenewer.applicationFinished(applicationId_1);
-    
+    waitForEventsToGetProcessed(delegationTokenRenewer);
     numberOfExpectedRenewals = Renewer.counter; // number of renewals so far
     numberOfExpectedRenewals = Renewer.counter; // number of renewals so far
     try {
     try {
       Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew
       Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew
@@ -377,8 +416,8 @@ public class TestDelegationTokenRenewer {
     }
     }
   }
   }
   
   
-  @Test
-  public void testInvalidDTWithAddApplication() throws Exception {
+  @Test(timeout=60000)
+  public void testAppRejectionWithCancelledDelegationToken() throws Exception {
     MyFS dfs = (MyFS)FileSystem.get(conf);
     MyFS dfs = (MyFS)FileSystem.get(conf);
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
 
 
@@ -390,12 +429,21 @@ public class TestDelegationTokenRenewer {
     
     
     // register the tokens for renewal
     // register the tokens for renewal
     ApplicationId appId =  BuilderUtils.newApplicationId(0, 0);
     ApplicationId appId =  BuilderUtils.newApplicationId(0, 0);
-    try {
-      delegationTokenRenewer.addApplication(appId, ts, true);
-      fail("App submission with a cancelled token should have failed");
-    } catch (InvalidToken e) {
-      // expected
+    delegationTokenRenewer.addApplication(appId, ts, true, false);
+    int waitCnt = 20;
+    while (waitCnt-- >0) {
+      if (!eventQueue.isEmpty()) {
+        Event evt = eventQueue.take();
+        if (evt.getType() == RMAppEventType.APP_REJECTED) {
+          Assert.assertTrue(
+              ((RMAppEvent) evt).getApplicationId().equals(appId));
+          return;
+        }
+      } else {
+        Thread.sleep(500);
+      }
     }
     }
+    fail("App submission with a cancelled token should have failed");
   }
   }
   
   
   /**
   /**
@@ -408,7 +456,7 @@ public class TestDelegationTokenRenewer {
    * @throws IOException
    * @throws IOException
    * @throws URISyntaxException
    * @throws URISyntaxException
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testDTRenewalWithNoCancel () throws Exception {
   public void testDTRenewalWithNoCancel () throws Exception {
     MyFS dfs = (MyFS)FileSystem.get(conf);
     MyFS dfs = (MyFS)FileSystem.get(conf);
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
@@ -425,9 +473,10 @@ public class TestDelegationTokenRenewer {
     
     
 
 
     ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
     ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
-    delegationTokenRenewer.addApplication(applicationId_1, ts, false);
+    delegationTokenRenewer.addApplication(applicationId_1, ts, false, false);
+    waitForEventsToGetProcessed(delegationTokenRenewer);
     delegationTokenRenewer.applicationFinished(applicationId_1);
     delegationTokenRenewer.applicationFinished(applicationId_1);
-    
+    waitForEventsToGetProcessed(delegationTokenRenewer);
     int numberOfExpectedRenewals = Renewer.counter; // number of renewals so far
     int numberOfExpectedRenewals = Renewer.counter; // number of renewals so far
     try {
     try {
       Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew
       Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew
@@ -454,9 +503,8 @@ public class TestDelegationTokenRenewer {
    * @throws IOException
    * @throws IOException
    * @throws URISyntaxException
    * @throws URISyntaxException
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testDTKeepAlive1 () throws Exception {
   public void testDTKeepAlive1 () throws Exception {
-    DelegationTokenRenewer localDtr = new DelegationTokenRenewer();
     Configuration lconf = new Configuration(conf);
     Configuration lconf = new Configuration(conf);
     lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     //Keep tokens alive for 6 seconds.
     //Keep tokens alive for 6 seconds.
@@ -465,10 +513,15 @@ public class TestDelegationTokenRenewer {
     lconf.setLong(
     lconf.setLong(
         YarnConfiguration.RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS,
         YarnConfiguration.RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS,
         1000l);
         1000l);
+    DelegationTokenRenewer localDtr =
+        createNewDelegationTokenRenewer(lconf, counter);
     localDtr.init(lconf);
     localDtr.init(lconf);
     RMContext mockContext = mock(RMContext.class);
     RMContext mockContext = mock(RMContext.class);
     ClientRMService mockClientRMService = mock(ClientRMService.class);
     ClientRMService mockClientRMService = mock(ClientRMService.class);
     when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
     when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
+    when(mockContext.getDelegationTokenRenewer()).thenReturn(
+        localDtr);
+    when(mockContext.getDispatcher()).thenReturn(dispatcher);
     InetSocketAddress sockAddr =
     InetSocketAddress sockAddr =
         InetSocketAddress.createUnresolved("localhost", 1234);
         InetSocketAddress.createUnresolved("localhost", 1234);
     when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
     when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
@@ -487,16 +540,25 @@ public class TestDelegationTokenRenewer {
 
 
     // register the tokens for renewal
     // register the tokens for renewal
     ApplicationId applicationId_0 =  BuilderUtils.newApplicationId(0, 0);
     ApplicationId applicationId_0 =  BuilderUtils.newApplicationId(0, 0);
-    localDtr.addApplication(applicationId_0, ts, true);
+    localDtr.addApplication(applicationId_0, ts, true, false);
+    waitForEventsToGetProcessed(localDtr);
+    if (!eventQueue.isEmpty()){
+      Event evt = eventQueue.take();
+      if (evt instanceof RMAppEvent) {
+        Assert.assertEquals(((RMAppEvent)evt).getType(), RMAppEventType.START);
+      } else {
+        fail("RMAppEvent.START was expected!!");
+      }
+    }
+    
     localDtr.applicationFinished(applicationId_0);
     localDtr.applicationFinished(applicationId_0);
- 
-    Thread.sleep(3000l);
+    waitForEventsToGetProcessed(localDtr);
 
 
     //Token should still be around. Renewal should not fail.
     //Token should still be around. Renewal should not fail.
     token1.renew(lconf);
     token1.renew(lconf);
 
 
     //Allow the keepalive time to run out
     //Allow the keepalive time to run out
-    Thread.sleep(6000l);
+    Thread.sleep(10000l);
 
 
     //The token should have been cancelled at this point. Renewal will fail.
     //The token should have been cancelled at this point. Renewal will fail.
     try {
     try {
@@ -518,9 +580,8 @@ public class TestDelegationTokenRenewer {
    * @throws IOException
    * @throws IOException
    * @throws URISyntaxException
    * @throws URISyntaxException
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testDTKeepAlive2() throws Exception {
   public void testDTKeepAlive2() throws Exception {
-    DelegationTokenRenewer localDtr = new DelegationTokenRenewer();
     Configuration lconf = new Configuration(conf);
     Configuration lconf = new Configuration(conf);
     lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     //Keep tokens alive for 6 seconds.
     //Keep tokens alive for 6 seconds.
@@ -529,10 +590,15 @@ public class TestDelegationTokenRenewer {
     lconf.setLong(
     lconf.setLong(
         YarnConfiguration.RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS,
         YarnConfiguration.RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS,
         1000l);
         1000l);
+    DelegationTokenRenewer localDtr =
+        createNewDelegationTokenRenewer(conf, counter);
     localDtr.init(lconf);
     localDtr.init(lconf);
     RMContext mockContext = mock(RMContext.class);
     RMContext mockContext = mock(RMContext.class);
     ClientRMService mockClientRMService = mock(ClientRMService.class);
     ClientRMService mockClientRMService = mock(ClientRMService.class);
     when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
     when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
+    when(mockContext.getDelegationTokenRenewer()).thenReturn(
+        localDtr);
+    when(mockContext.getDispatcher()).thenReturn(dispatcher);
     InetSocketAddress sockAddr =
     InetSocketAddress sockAddr =
         InetSocketAddress.createUnresolved("localhost", 1234);
         InetSocketAddress.createUnresolved("localhost", 1234);
     when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
     when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
@@ -551,22 +617,18 @@ public class TestDelegationTokenRenewer {
 
 
     // register the tokens for renewal
     // register the tokens for renewal
     ApplicationId applicationId_0 =  BuilderUtils.newApplicationId(0, 0);
     ApplicationId applicationId_0 =  BuilderUtils.newApplicationId(0, 0);
-    localDtr.addApplication(applicationId_0, ts, true);
+    localDtr.addApplication(applicationId_0, ts, true, false);
     localDtr.applicationFinished(applicationId_0);
     localDtr.applicationFinished(applicationId_0);
-
-    Thread.sleep(4000l);
-
+    waitForEventsToGetProcessed(delegationTokenRenewer);
     //Send another keep alive.
     //Send another keep alive.
     localDtr.updateKeepAliveApplications(Collections
     localDtr.updateKeepAliveApplications(Collections
         .singletonList(applicationId_0));
         .singletonList(applicationId_0));
     //Renewal should not fail.
     //Renewal should not fail.
     token1.renew(lconf);
     token1.renew(lconf);
-
     //Token should be around after this. 
     //Token should be around after this. 
     Thread.sleep(4500l);
     Thread.sleep(4500l);
     //Renewal should not fail. - ~1.5 seconds for keepalive timeout.
     //Renewal should not fail. - ~1.5 seconds for keepalive timeout.
     token1.renew(lconf);
     token1.renew(lconf);
-
     //Allow the keepalive time to run out
     //Allow the keepalive time to run out
     Thread.sleep(3000l);
     Thread.sleep(3000l);
     //The token should have been cancelled at this point. Renewal will fail.
     //The token should have been cancelled at this point. Renewal will fail.
@@ -575,61 +637,127 @@ public class TestDelegationTokenRenewer {
       fail("Renewal of cancelled token should have failed");
       fail("Renewal of cancelled token should have failed");
     } catch (InvalidToken ite) {}
     } catch (InvalidToken ite) {}
   }
   }
-  
-  @Test(timeout=20000)
-  public void testConncurrentAddApplication()
-      throws IOException, InterruptedException, BrokenBarrierException {
-    final CyclicBarrier startBarrier = new CyclicBarrier(2);
-    final CyclicBarrier endBarrier = new CyclicBarrier(2);
-
-    // this token uses barriers to block during renew
-    final Credentials creds1 = new Credentials();
-    final Token<?> token1 = mock(Token.class);
-    creds1.addToken(new Text("token"), token1);
-    doReturn(true).when(token1).isManaged();
-    doAnswer(new Answer<Long>() {
-      public Long answer(InvocationOnMock invocation)
-          throws InterruptedException, BrokenBarrierException {
-        startBarrier.await();
-        endBarrier.await();
-        return Long.MAX_VALUE;
-      }}).when(token1).renew(any(Configuration.class));
-
-    // this dummy token fakes renewing
-    final Credentials creds2 = new Credentials();
-    final Token<?> token2 = mock(Token.class);
-    creds2.addToken(new Text("token"), token2);
-    doReturn(true).when(token2).isManaged();
-    doReturn(Long.MAX_VALUE).when(token2).renew(any(Configuration.class));
-
-    // fire up the renewer
-    final DelegationTokenRenewer dtr = new DelegationTokenRenewer();
-    dtr.init(conf);
-    RMContext mockContext = mock(RMContext.class);
-    ClientRMService mockClientRMService = mock(ClientRMService.class);
-    when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
-    InetSocketAddress sockAddr =
-        InetSocketAddress.createUnresolved("localhost", 1234);
-    when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
-    dtr.setRMContext(mockContext);
-    dtr.start();
-    
-    // submit a job that blocks during renewal
-    Thread submitThread = new Thread() {
+
+  private DelegationTokenRenewer createNewDelegationTokenRenewer(
+      Configuration conf, final AtomicInteger counter) {
+    return new DelegationTokenRenewer() {
+
       @Override
       @Override
-      public void run() {
-        try {
-          dtr.addApplication(mock(ApplicationId.class), creds1, false);
-        } catch (IOException e) {}        
+      protected ThreadPoolExecutor
+          createNewThreadPoolService(Configuration conf) {
+        ThreadPoolExecutor pool =
+            new ThreadPoolExecutor(5, 5, 3L, TimeUnit.SECONDS,
+                new LinkedBlockingQueue<Runnable>()) {
+
+              @Override
+              protected void afterExecute(Runnable r, Throwable t) {
+                counter.decrementAndGet();
+                super.afterExecute(r, t);
+              }
+
+              @Override
+              public void execute(Runnable command) {
+                counter.incrementAndGet();
+                super.execute(command);
+              }
+            };
+        return pool;
       }
       }
     };
     };
-    submitThread.start();
-    
+  }
+
+  private void waitForEventsToGetProcessed(DelegationTokenRenewer dtr)
+      throws InterruptedException {
+    int wait = 40;
+    while (wait-- > 0
+        && counter.get() > 0) {
+      Thread.sleep(200);
+    }
+  }
+  
+  @Test(timeout=20000)                                                         
+  public void testConcurrentAddApplication()                                  
+      throws IOException, InterruptedException, BrokenBarrierException {       
+    final CyclicBarrier startBarrier = new CyclicBarrier(2);                   
+    final CyclicBarrier endBarrier = new CyclicBarrier(2);                     
+                                                                               
+    // this token uses barriers to block during renew                          
+    final Credentials creds1 = new Credentials();                              
+    final Token<?> token1 = mock(Token.class);                                 
+    creds1.addToken(new Text("token"), token1);                                
+    doReturn(true).when(token1).isManaged();                                   
+    doAnswer(new Answer<Long>() {                                              
+      public Long answer(InvocationOnMock invocation)                          
+          throws InterruptedException, BrokenBarrierException { 
+        startBarrier.await();                                                  
+        endBarrier.await();                                                    
+        return Long.MAX_VALUE;                                                 
+      }}).when(token1).renew(any(Configuration.class));                        
+                                                                               
+    // this dummy token fakes renewing                                         
+    final Credentials creds2 = new Credentials();                              
+    final Token<?> token2 = mock(Token.class);                                 
+    creds2.addToken(new Text("token"), token2);                                
+    doReturn(true).when(token2).isManaged();                                   
+    doReturn(Long.MAX_VALUE).when(token2).renew(any(Configuration.class));     
+                                                                               
+    // fire up the renewer                                                     
+    final DelegationTokenRenewer dtr =
+        createNewDelegationTokenRenewer(conf, counter);           
+    dtr.init(conf);                                                            
+    RMContext mockContext = mock(RMContext.class);                             
+    ClientRMService mockClientRMService = mock(ClientRMService.class);         
+    when(mockContext.getClientRMService()).thenReturn(mockClientRMService);    
+    InetSocketAddress sockAddr =                                               
+        InetSocketAddress.createUnresolved("localhost", 1234);                 
+    when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);           
+    dtr.setRMContext(mockContext);  
+    when(mockContext.getDelegationTokenRenewer()).thenReturn(dtr);
+    dtr.start();                                                                           
+    // submit a job that blocks during renewal                                 
+    Thread submitThread = new Thread() {                                       
+      @Override                                                                
+      public void run() {
+        dtr.addApplication(mock(ApplicationId.class), creds1, false, false);        
+      }                                                                        
+    };                                                                         
+    submitThread.start();                                                      
+                                                                               
     // wait till 1st submit blocks, then submit another
     // wait till 1st submit blocks, then submit another
-    startBarrier.await();
-    dtr.addApplication(mock(ApplicationId.class), creds2, false);
-    // signal 1st to complete
-    endBarrier.await();
-    submitThread.join();
+    startBarrier.await();                           
+    dtr.addApplication(mock(ApplicationId.class), creds2, false, false);              
+    // signal 1st to complete                                                  
+    endBarrier.await();                                                        
+    submitThread.join(); 
+  }
+  
+  @Test(timeout=20000)
+  public void testAppSubmissionWithInvalidDelegationToken() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    MockRM rm = new MockRM(conf);
+    ByteBuffer tokens = ByteBuffer.wrap("BOGUS".getBytes()); 
+    ContainerLaunchContext amContainer =
+        ContainerLaunchContext.newInstance(
+            new HashMap<String, LocalResource>(), new HashMap<String, String>(),
+            new ArrayList<String>(), new HashMap<String, ByteBuffer>(), tokens,
+            new HashMap<ApplicationAccessType, String>());
+    ApplicationSubmissionContext appSubContext =
+        ApplicationSubmissionContext.newInstance(
+            ApplicationId.newInstance(1234121, 0),
+            "BOGUS", "default", Priority.UNDEFINED, amContainer, false,
+            true, 1, Resource.newInstance(1024, 1), "BOGUS");
+    SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(appSubContext);
+    try {
+      rm.getClientRMService().submitApplication(request);
+      fail("Error was excepted.");
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Bad header found in token storage"));
+    }
   }
   }
 }
 }

Some files were not shown because too many files changed in this diff