Browse Source

Merging r1536890 through r1537251 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1537252 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 years ago
parent
commit
4eebf56ccf
36 changed files with 660 additions and 204 deletions
  1. 12 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  4. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
  5. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  6. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  8. 15 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java
  9. 25 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java
  10. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HsftpFileSystem.java
  11. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  12. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
  14. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  15. 300 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/CentralizedCacheManagement.apt.vm
  16. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/caching.png
  17. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  18. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  19. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
  20. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListPathServlet.java
  21. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
  23. 131 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestCheckpointsWithSnapshots.java
  24. 18 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
  25. 24 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpDelegationToken.java
  26. 30 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpFileSystem.java
  27. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpURLTimeouts.java
  28. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java
  29. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
  30. 1 0
      hadoop-project/src/site/site.xml
  31. 0 1
      hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java
  32. 0 7
      hadoop-tools/hadoop-sls/src/main/sample-conf/capacity-scheduler.xml
  33. 0 50
      hadoop-tools/hadoop-sls/src/main/sample-conf/fair-scheduler-allocation.xml
  34. 28 25
      hadoop-tools/hadoop-sls/src/main/sample-conf/fair-scheduler.xml
  35. 20 0
      hadoop-tools/hadoop-sls/src/main/sample-conf/yarn-site.xml
  36. 3 0
      hadoop-yarn-project/CHANGES.txt

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

@@ -188,6 +188,9 @@ Trunk (Unreleased)
     HDFS-5378. In CacheReport, don't send genstamp and length on the wire
     (Contributed by Colin Patrick McCabe)
 
+    HDFS-5386. Add feature documentation for datanode caching.
+    (Colin Patrick McCabe via cnauroth)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
@@ -447,6 +450,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5363. Refactor WebHdfsFileSystem: move SPENGO-authenticated connection
     creation to URLConnectionFactory. (Haohui Mai via jing9)
 
+    HDFS-5436. Move HsFtpFileSystem and HFtpFileSystem into org.apache.hdfs.web
+    (Haohui Mai via Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -562,6 +568,12 @@ Release 2.2.1 - UNRELEASED
     HDFS-5413. hdfs.cmd does not support passthrough to any arbitrary class.
     (cnauroth)
 
+    HDFS-5433. When reloading fsimage during checkpointing, we should clear
+    existing snapshottable directories. (Aaron T. Myers via wang)
+
+    HDFS-5432. TestDatanodeJsp fails on Windows due to assumption that loopback
+    address resolves to host name localhost. (cnauroth)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -513,6 +513,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
     leaseManager.removeAllLeases();
     inodeId.setCurrentValue(INodeId.LAST_RESERVED_ID);
+    snapshotManager.clearSnapshottableDirs();
   }
 
   @VisibleForTesting

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

@@ -38,7 +38,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ServletUtil;
 
 /** Redirect queries about the hosted filesystem to an appropriate datanode.
- * @see org.apache.hadoop.hdfs.HftpFileSystem
+ * @see org.apache.hadoop.hdfs.web.HftpFileSystem
  */
 @InterfaceAudience.Private
 public class FileDataServlet extends DfsServlet {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java

@@ -20,14 +20,13 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.VersionInfo;
-
 import org.znerd.xmlenc.*;
 
 import java.io.IOException;
@@ -39,13 +38,14 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Stack;
 import java.util.regex.Pattern;
+
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 /**
  * Obtain meta-information about a filesystem.
- * @see org.apache.hadoop.hdfs.HftpFileSystem
+ * @see org.apache.hadoop.hdfs.web.HftpFileSystem
  */
 @InterfaceAudience.Private
 public class ListPathsServlet extends DfsServlet {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -155,7 +155,7 @@ public class SecondaryNameNode implements Runnable {
   }
 
   @VisibleForTesting
-  FSNamesystem getFSNamesystem() {
+  public FSNamesystem getFSNamesystem() {
     return namesystem;
   }
   
@@ -489,7 +489,8 @@ public class SecondaryNameNode implements Runnable {
    * Create a new checkpoint
    * @return if the image is fetched from primary or not
    */
-  boolean doCheckpoint() throws IOException {
+  @VisibleForTesting
+  public boolean doCheckpoint() throws IOException {
     checkpointImage.ensureCurrentDirExists();
     NNStorage dstStorage = checkpointImage.getStorage();
     

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -357,6 +357,10 @@ public class SnapshotManager implements SnapshotStats {
     
     return snapshotRoot.computeDiff(from, to);
   }
+  
+  public void clearSnapshottableDirs() {
+    snapshottables.clear();
+  }
 
   /**
    * Returns the maximum allowable snapshot ID based on the bit width of the

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -41,12 +41,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;

+ 15 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -34,28 +34,28 @@ import com.google.common.net.HttpHeaders;
 
 /**
  * To support HTTP byte streams, a new connection to an HTTP server needs to be
- * created each time. This class hides the complexity of those multiple 
+ * created each time. This class hides the complexity of those multiple
  * connections from the client. Whenever seek() is called, a new connection
- * is made on the successive read(). The normal input stream functions are 
- * connected to the currently active input stream. 
+ * is made on the successive read(). The normal input stream functions are
+ * connected to the currently active input stream.
  */
 public abstract class ByteRangeInputStream extends FSInputStream {
-  
+
   /**
    * This class wraps a URL and provides method to open connection.
    * It can be overridden to change how a connection is opened.
    */
   public static abstract class URLOpener {
     protected URL url;
-  
+
     public URLOpener(URL u) {
       url = u;
     }
-  
+
     public void setURL(URL u) {
       url = u;
     }
-  
+
     public URL getURL() {
       return url;
     }
@@ -78,7 +78,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
   StreamStatus status = StreamStatus.SEEK;
 
   /**
-   * Create with the specified URLOpeners. Original url is used to open the 
+   * Create with the specified URLOpeners. Original url is used to open the
    * stream for the first time. Resolved url is used in subsequent requests.
    * @param o Original url
    * @param r Resolved url
@@ -87,7 +87,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
     this.originalURL = o;
     this.resolvedURL = r;
   }
-  
+
   protected abstract URL getResolvedUrl(final HttpURLConnection connection
       ) throws IOException;
 
@@ -108,12 +108,12 @@ public abstract class ByteRangeInputStream extends FSInputStream {
     }
     return in;
   }
-  
+
   @VisibleForTesting
   protected InputStream openInputStream() throws IOException {
     // Use the original url if no resolved url exists, eg. if
     // it's the first time a request is made.
-    final boolean resolved = resolvedURL.getURL() != null; 
+    final boolean resolved = resolvedURL.getURL() != null;
     final URLOpener opener = resolved? resolvedURL: originalURL;
 
     final HttpURLConnection connection = opener.connect(startPos, resolved);
@@ -141,7 +141,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
 
     return in;
   }
-  
+
   private static boolean isChunkedTransferEncoding(
       final Map<String, List<String>> headers) {
     return contains(headers, HttpHeaders.TRANSFER_ENCODING, "chunked")
@@ -186,7 +186,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
   public int read(byte b[], int off, int len) throws IOException {
     return update(getInputStream().read(b, off, len));
   }
-  
+
   /**
    * Seek to the given offset from the start of the file.
    * The next read() will be from that location.  Can't
@@ -219,7 +219,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
   public boolean seekToNewSource(long targetPos) throws IOException {
     return false;
   }
-  
+
   @Override
   public void close() throws IOException {
     if (in != null) {

+ 25 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -47,11 +47,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
-import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.hdfs.web.ByteRangeInputStream.URLOpener;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -161,7 +163,7 @@ public class HftpFileSystem extends FileSystem
   public String getCanonicalServiceName() {
     return SecurityUtil.buildTokenService(nnUri).toString();
   }
-  
+
   @Override
   protected URI canonicalizeUri(URI uri) {
     return NetUtils.getCanonicalUri(uri, getDefaultPort());
@@ -183,7 +185,7 @@ public class HftpFileSystem extends FileSystem
   throws IOException {
     super.initialize(name, conf);
     setConf(conf);
-    this.ugi = UserGroupInformation.getCurrentUser(); 
+    this.ugi = UserGroupInformation.getCurrentUser();
     this.nnUri = getNamenodeUri(name);
     try {
       this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
@@ -224,7 +226,7 @@ public class HftpFileSystem extends FileSystem
       UserGroupInformation ugi) {
     return hftpTokenSelector.selectToken(nnUri, ugi.getTokens(), getConf());
   }
-  
+
 
   @Override
   public Token<?> getRenewToken() {
@@ -315,7 +317,7 @@ public class HftpFileSystem extends FileSystem
 
   /**
    * Get encoded UGI parameter string for a URL.
-   * 
+   *
    * @return user_shortname,group1,group2...
    */
   private String getEncodedUgiParameter() {
@@ -359,7 +361,7 @@ public class HftpFileSystem extends FileSystem
 
   static class RangeHeaderUrlOpener extends ByteRangeInputStream.URLOpener {
     URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
-    
+
     RangeHeaderUrlOpener(final URL url) {
       super(url);
     }
@@ -379,7 +381,7 @@ public class HftpFileSystem extends FileSystem
       }
       conn.connect();
 
-      //Expects HTTP_OK or HTTP_PARTIAL response codes. 
+      //Expects HTTP_OK or HTTP_PARTIAL response codes.
       final int code = conn.getResponseCode();
       if (offset != 0L && code != HttpURLConnection.HTTP_PARTIAL) {
         throw new IOException("HTTP_PARTIAL expected, received " + code);
@@ -387,7 +389,7 @@ public class HftpFileSystem extends FileSystem
         throw new IOException("HTTP_OK expected, received " + code);
       }
       return conn;
-    }  
+    }
   }
 
   static class RangeHeaderInputStream extends ByteRangeInputStream {
@@ -410,7 +412,7 @@ public class HftpFileSystem extends FileSystem
     f = f.makeQualified(getUri(), getWorkingDirectory());
     String path = "/data" + ServletUtil.encodePath(f.toUri().getPath());
     String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter());
-    URL u = getNamenodeURL(path, query);    
+    URL u = getNamenodeURL(path, query);
     return new FSDataInputStream(new RangeHeaderInputStream(u));
   }
 
@@ -533,7 +535,7 @@ public class HftpFileSystem extends FileSystem
 
     private FileChecksum getFileChecksum(String f) throws IOException {
       final HttpURLConnection connection = openConnection(
-          "/fileChecksum" + ServletUtil.encodePath(f), 
+          "/fileChecksum" + ServletUtil.encodePath(f),
           "ugi=" + getEncodedUgiParameter());
       try {
         final XMLReader xr = XMLReaderFactory.createXMLReader();
@@ -585,11 +587,11 @@ public class HftpFileSystem extends FileSystem
     throw new IOException("Not supported");
   }
 
-  @Override 
+  @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
     throw new IOException("Not supported");
   }
-  
+
   @Override
   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
     throw new IOException("Not supported");
@@ -615,18 +617,18 @@ public class HftpFileSystem extends FileSystem
     }
 
     /**
-     * Connect to the name node and get content summary.  
+     * Connect to the name node and get content summary.
      * @param path The path
      * @return The content summary for the path.
      * @throws IOException
      */
     private ContentSummary getContentSummary(String path) throws IOException {
       final HttpURLConnection connection = openConnection(
-          "/contentSummary" + ServletUtil.encodePath(path), 
+          "/contentSummary" + ServletUtil.encodePath(path),
           "ugi=" + getEncodedUgiParameter());
       InputStream in = null;
       try {
-        in = connection.getInputStream();        
+        in = connection.getInputStream();
 
         final XMLReader xr = XMLReaderFactory.createXMLReader();
         xr.setContentHandler(this);
@@ -713,12 +715,12 @@ public class HftpFileSystem extends FileSystem
 
     @SuppressWarnings("unchecked")
     @Override
-    public long renew(Token<?> token, 
+    public long renew(Token<?> token,
                       Configuration conf) throws IOException {
       // update the kerberos credentials, if they are coming from a keytab
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
       InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
-      return 
+      return
         DelegationTokenFetcher.renewDelegationToken
         (DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
          (Token<DelegationTokenIdentifier>) token);
@@ -726,7 +728,7 @@ public class HftpFileSystem extends FileSystem
 
     @SuppressWarnings("unchecked")
     @Override
-    public void cancel(Token<?> token, 
+    public void cancel(Token<?> token,
                        Configuration conf) throws IOException {
       // update the kerberos credentials, if they are coming from a keytab
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
@@ -734,9 +736,9 @@ public class HftpFileSystem extends FileSystem
       DelegationTokenFetcher.cancelDelegationToken
         (DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
          (Token<DelegationTokenIdentifier>) token);
-    }    
+    }
   }
-  
+
   private static class HftpDelegationTokenSelector
   extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
     private static final DelegationTokenSelector hdfsTokenSelector =
@@ -745,14 +747,14 @@ public class HftpFileSystem extends FileSystem
     public HftpDelegationTokenSelector() {
       super(TOKEN_KIND);
     }
-    
+
     Token<DelegationTokenIdentifier> selectToken(URI nnUri,
         Collection<Token<?>> tokens, Configuration conf) {
       Token<DelegationTokenIdentifier> token =
           selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
       if (token == null) {
         // try to get a HDFS token
-        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf); 
+        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf);
       }
       return token;
     }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HsftpFileSystem.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import java.io.FileInputStream;
 import java.io.IOException;
@@ -40,13 +40,15 @@ import javax.net.ssl.X509TrustManager;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.util.Time;
 
 /**
  * An implementation of a protocol for accessing filesystems over HTTPS. The
  * following implementation provides a limited, read-only interface to a
  * filesystem over HTTPS.
- * 
+ *
  * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
  * @see org.apache.hadoop.hdfs.server.namenode.FileDataServlet
  */
@@ -85,7 +87,7 @@ public class HsftpFileSystem extends HftpFileSystem {
 
   /**
    * Set up SSL resources
-   * 
+   *
    * @throws IOException
    */
   private static void setupSsl(Configuration conf) throws IOException {

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -51,7 +51,6 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem

@@ -14,6 +14,6 @@
 # limitations under the License.
 
 org.apache.hadoop.hdfs.DistributedFileSystem
-org.apache.hadoop.hdfs.HftpFileSystem
-org.apache.hadoop.hdfs.HsftpFileSystem
+org.apache.hadoop.hdfs.web.HftpFileSystem
+org.apache.hadoop.hdfs.web.HsftpFileSystem
 org.apache.hadoop.hdfs.web.WebHdfsFileSystem

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer

@@ -13,5 +13,5 @@
 #
 org.apache.hadoop.hdfs.DFSClient$Renewer
 org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
-org.apache.hadoop.hdfs.HftpFileSystem$TokenManager
+org.apache.hadoop.hdfs.web.HftpFileSystem$TokenManager
 org.apache.hadoop.hdfs.web.WebHdfsFileSystem$DtRenewer

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -1486,6 +1486,24 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.list.cache.descriptors.num.responses</name>
+  <value>100</value>
+  <description>
+    This value controls the number of cache descriptors that the NameNode will
+    send over the wire in response to a listDirectives RPC.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.list.cache.pools.num.responses</name>
+  <value>100</value>
+  <description>
+    This value controls the number of cache pools that the NameNode will
+    send over the wire in response to a listPools RPC.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.path.based.cache.refresh.interval.ms</name>
   <value>300000</value>

+ 300 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/CentralizedCacheManagement.apt.vm

@@ -0,0 +1,300 @@
+~~ Licensed under the Apache License, Version 2.0 (the "License");
+~~ you may not use this file except in compliance with the License.
+~~ You may obtain a copy of the License at
+~~
+~~   http://www.apache.org/licenses/LICENSE-2.0
+~~
+~~ Unless required by applicable law or agreed to in writing, software
+~~ distributed under the License is distributed on an "AS IS" BASIS,
+~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~~ See the License for the specific language governing permissions and
+~~ limitations under the License. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - Centralized Cache Management in HDFS
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Centralized Cache Management in HDFS
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=2|toDepth=4}
+
+* {Background}
+
+  Normally, HDFS relies on the operating system to cache data it reads from disk.
+  However, HDFS can also be configured to use centralized cache management. Under
+  centralized cache management, the HDFS NameNode itself decides which blocks
+  should be cached, and where they should be cached.
+
+  Centralized cache management has several advantages. First of all, it
+  prevents frequently used block files from being evicted from memory. This is
+  particularly important when the size of the working set exceeds the size of
+  main memory, which is true for many big data applications. Secondly, when
+  HDFS decides what should be cached, it can let clients know about this
+  information through the getFileBlockLocations API. Finally, when the DataNode
+  knows a block is locked into memory, it can provide access to that block via
+  mmap.
+
+* {Use Cases}
+
+  Centralized cache management is most useful for files which are accessed very
+  often. For example, a "fact table" in Hive which is often used in joins is a
+  good candidate for caching. On the other hand, when running a classic
+  "word count" MapReduce job which counts the number of words in each
+  document, there may not be any good candidates for caching, since all the
+  files may be accessed exactly once.
+
+* {Architecture}
+
+[images/caching.png] Caching Architecture
+
+  With centralized cache management, the NameNode coordinates all caching
+  across the cluster. It receives cache information from each DataNode via the
+  cache report, a periodic message that describes all the blocks IDs cached on
+  a given DataNode. The NameNode will reply to DataNode heartbeat messages
+  with commands telling it which blocks to cache and which to uncache.
+
+  The NameNode stores a set of path cache directives, which tell it which files
+  to cache. The NameNode also stores a set of cache pools, which are groups of
+  cache directives.  These directives and pools are persisted to the edit log
+  and fsimage, and will be loaded if the cluster is restarted.
+
+  Periodically, the NameNode rescans the namespace, to see which blocks need to
+  be cached based on the current set of path cache directives. Rescans are also
+  triggered by relevant user actions, such as adding or removing a cache
+  directive or removing a cache pool.
+
+  Cache directives also may specific a numeric cache replication, which is the
+  number of replicas to cache.  This number may be equal to or smaller than the
+  file's block replication.  If multiple cache directives cover the same file
+  with different cache replication settings, then the highest cache replication
+  setting is applied.
+
+  We do not currently cache blocks which are under construction, corrupt, or
+  otherwise incomplete.  If a cache directive covers a symlink, the symlink
+  target is not cached.
+
+  Caching is currently done on a per-file basis, although we would like to add
+  block-level granularity in the future.
+
+* {Interface}
+
+  The NameNode stores a list of "cache directives."  These directives contain a
+  path as well as the number of times blocks in that path should be replicated.
+
+  Paths can be either directories or files. If the path specifies a file, that
+  file is cached. If the path specifies a directory, all the files in the
+  directory will be cached. However, this process is not recursive-- only the
+  direct children of the directory will be cached.
+
+** {hdfs cacheadmin Shell}
+
+  Path cache directives can be created by the <<<hdfs cacheadmin
+  -addDirective>>> command and removed via the <<<hdfs cacheadmin
+  -removeDirective>>> command. To list the current path cache directives, use
+  <<<hdfs cacheadmin -listDirectives>>>. Each path cache directive has a
+  unique 64-bit ID number which will not be reused if it is deleted.  To remove
+  all path cache directives with a specified path, use <<<hdfs cacheadmin
+  -removeDirectives>>>.
+
+  Directives are grouped into "cache pools."  Each cache pool gets a share of
+  the cluster's resources. Additionally, cache pools are used for
+  authentication. Cache pools have a mode, user, and group, similar to regular
+  files. The same authentication rules are applied as for normal files. So, for
+  example, if the mode is 0777, any user can add or remove directives from the
+  cache pool. If the mode is 0644, only the owner can write to the cache pool,
+  but anyone can read from it. And so forth.
+
+  Cache pools are identified by name. They can be created by the <<<hdfs
+  cacheAdmin -addPool>>> command, modified by the <<<hdfs cacheadmin
+  -modifyPool>>> command, and removed via the <<<hdfs cacheadmin
+  -removePool>>> command. To list the current cache pools, use <<<hdfs
+  cacheAdmin -listPools>>>
+
+*** {addDirective}
+
+  Usage: <<<hdfs cacheadmin -addDirective -path <path> -replication <replication> -pool <pool-name> >>>
+
+  Add a new PathBasedCache directive.
+
+*--+--+
+\<path\> | A path to cache. The path can be a directory or a file.
+*--+--+
+\<replication\> | The cache replication factor to use. Defaults to 1.
+*--+--+
+\<pool-name\> | The pool to which the directive will be added. You must have write permission on the cache pool in order to add new directives.
+*--+--+
+
+*** {removeDirective}
+
+  Usage: <<<hdfs cacheadmin -removeDirective <id> >>>
+
+  Remove a cache directive.
+
+*--+--+
+\<id\> | The id of the cache directive to remove.  You must have write permission on the pool of the directive in order to remove it.  To see a list of PathBasedCache directive IDs, use the -listDirectives command.
+*--+--+
+
+*** {removeDirectives}
+
+  Usage: <<<hdfs cacheadmin -removeDirectives <path> >>>
+
+  Remove every cache directive with the specified path.
+
+*--+--+
+\<path\> | The path of the cache directives to remove.  You must have write permission on the pool of the directive in order to remove it.  To see a list of cache directives, use the -listDirectives command.
+*--+--+
+
+*** {listDirectives}
+
+  Usage: <<<hdfs cacheadmin -listDirectives [-path <path>] [-pool <pool>] >>>
+
+  List PathBasedCache directives.
+
+*--+--+
+\<path\> | List only PathBasedCache directives with this path. Note that if there is a PathBasedCache directive for <path> in a cache pool that we don't have read access for, it will not be listed.
+*--+--+
+\<pool\> | List only path cache directives in that pool.
+*--+--+
+
+*** {addPool}
+
+  Usage: <<<hdfs cacheadmin -addPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-weight <weight>] >>>
+
+  Add a new cache pool.
+
+*--+--+
+\<name\> | Name of the new pool.
+*--+--+
+\<owner\> | Username of the owner of the pool. Defaults to the current user.
+*--+--+
+\<group\> | Group of the pool. Defaults to the primary group name of the current user.
+*--+--+
+\<mode\> | UNIX-style permissions for the pool. Permissions are specified in octal, e.g. 0755. By default, this is set to 0755.
+*--+--+
+\<weight\> | Weight of the pool. This is a relative measure of the importance of the pool used during cache resource management. By default, it is set to 100.
+*--+--+
+
+*** {modifyPool}
+
+  Usage: <<<hdfs cacheadmin -modifyPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-weight <weight>] >>>
+
+  Modifies the metadata of an existing cache pool.
+
+*--+--+
+\<name\> | Name of the pool to modify.
+*--+--+
+\<owner\> | Username of the owner of the pool.
+*--+--+
+\<group\> | Groupname of the group of the pool.
+*--+--+
+\<mode\> | Unix-style permissions of the pool in octal.
+*--+--+
+\<weight\> | Weight of the pool.
+*--+--+
+
+*** {removePool}
+
+  Usage: <<<hdfs cacheadmin -removePool <name> >>>
+
+  Remove a cache pool. This also uncaches paths associated with the pool.
+
+*--+--+
+\<name\> | Name of the cache pool to remove.
+*--+--+
+
+*** {listPools}
+
+  Usage: <<<hdfs cacheadmin -listPools [name] >>>
+
+  Display information about one or more cache pools, e.g. name, owner, group,
+  permissions, etc.
+
+*--+--+
+\<name\> | If specified, list only the named cache pool.
+*--+--+
+
+*** {help}
+
+  Usage: <<<hdfs cacheadmin -help <command-name> >>>
+
+  Get detailed help about a command.
+
+*--+--+
+\<command-name\> | The command for which to get detailed help. If no command is specified, print detailed help for all commands.
+*--+--+
+
+* {Configuration}
+
+** {Native Libraries}
+
+  In order to lock block files into memory, the DataNode relies on native JNI
+  code found in <<<libhadoop.so>>>. Be sure to
+  {{{../hadoop-common/NativeLibraries.html}enable JNI}} if you are using HDFS
+  centralized cache management.
+
+** {Configuration Properties}
+
+*** Required
+
+  Be sure to configure the following:
+
+  * dfs.namenode.caching.enabled
+
+    This must be set to true to enable caching. If this is false, the NameNode
+    will ignore cache reports, and will not ask DataNodes to cache
+    blocks.
+
+  * dfs.datanode.max.locked.memory
+
+    The DataNode will treat this as the maximum amount of memory it can use for
+    its cache. When setting this value, please remember that you will need space
+    in memory for other things, such as the Java virtual machine (JVM) itself
+    and the operating system's page cache.
+
+*** Optional
+
+  The following properties are not required, but may be specified for tuning:
+
+  * dfs.namenode.path.based.cache.refresh.interval.ms
+
+    The NameNode will use this as the amount of milliseconds between subsequent
+    path cache rescans.  This calculates the blocks to cache and each DataNode
+    containing a replica of the block that should cache it.
+
+    By default, this parameter is set to 300000, which is five minutes.
+
+  * dfs.datanode.fsdatasetcache.max.threads.per.volume
+
+    The DataNode will use this as the maximum number of threads per volume to
+    use for caching new data.
+
+    By default, this parameter is set to 4.
+
+  * dfs.cachereport.intervalMsec
+
+    The DataNode will use this as the amount of milliseconds between sending a
+    full report of its cache state to the NameNode.
+
+    By default, this parameter is set to 10000, which is 10 seconds.
+
+** {OS Limits}
+
+  If you get the error "Cannot start datanode because the configured max
+  locked memory size... is more than the datanode's available RLIMIT_MEMLOCK
+  ulimit," that means that the operating system is imposing a lower limit
+  on the amount of memory that you can lock than what you have configured. To
+  fix this, you must adjust the ulimit -l value that the DataNode runs with.
+  Usually, this value is configured in <<</etc/security/limits.conf>>>.
+  However, it will vary depending on what operating system and distribution
+  you are using.
+
+  You will know that you have correctly configured this value when you can run
+  <<<ulimit -l>>> from the shell and get back either a higher value than what
+  you have configured with <<<dfs.datanode.max.locked.memory>>>, or the string
+  "unlimited," indicating that there is no limit.  Note that it's typical for
+  <<<ulimit -l>>> to output the memory lock limit in KB, but
+  dfs.datanode.max.locked.memory must be specified in bytes.

BIN
hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/caching.png


+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -89,6 +89,7 @@ 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.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -56,6 +56,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.DataChecksum;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListPathServlet.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.namenode.ListPathsServlet;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
 import org.junit.Assert;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java

@@ -88,7 +88,8 @@ public class TestDatanodeJsp {
     regex = "<a.+href=\"(.+?)\">Go\\s*Back\\s*to\\s*File\\s*View\\<\\/a\\>";
     assertFileContents(regex, "Go Back to File View");
 
-    regex = "<a href=\"///localhost:" + nnHttpAddress.getPort() + "/dfshealth.jsp\">Go back to DFS home</a>";
+    regex = "<a href=\"///" + nnHttpAddress.getHostName() + ":" +
+      nnHttpAddress.getPort() + "/dfshealth.jsp\">Go back to DFS home</a>";
     assertTrue("page should generate DFS home scheme without explicit scheme", viewFilePage.contains(regex));
   }
   

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

@@ -39,8 +39,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.AccessControlException;

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

@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCheckpointsWithSnapshots {
+  
+  private static final Path TEST_PATH = new Path("/foo");
+  private static Configuration conf = new HdfsConfiguration();
+  static {
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+  }
+  
+  @Before
+  public void setUp() {
+    FileUtil.fullyDeleteContents(new File(MiniDFSCluster.getBaseDirectory()));
+  }
+
+  /**
+   * Regression test for HDFS-5433 - "When reloading fsimage during
+   * checkpointing, we should clear existing snapshottable directories"
+   */
+  @Test
+  public void testCheckpoint() throws IOException {
+    MiniDFSCluster cluster = null;
+    SecondaryNameNode secondary = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      secondary = new SecondaryNameNode(conf);
+      SnapshotManager nnSnapshotManager = cluster.getNamesystem().getSnapshotManager();
+      SnapshotManager secondarySnapshotManager = secondary.getFSNamesystem().getSnapshotManager();
+      
+      FileSystem fs = cluster.getFileSystem();
+      HdfsAdmin admin =  new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+      
+      assertEquals(0, nnSnapshotManager.getNumSnapshots());
+      assertEquals(0, nnSnapshotManager.getNumSnapshottableDirs());
+      assertEquals(0, secondarySnapshotManager.getNumSnapshots());
+      assertEquals(0, secondarySnapshotManager.getNumSnapshottableDirs());
+      
+      // 1. Create a snapshottable directory foo on the NN.
+      fs.mkdirs(TEST_PATH);
+      admin.allowSnapshot(TEST_PATH);
+      assertEquals(0, nnSnapshotManager.getNumSnapshots());
+      assertEquals(1, nnSnapshotManager.getNumSnapshottableDirs());
+      
+      // 2. Create a snapshot of the dir foo. This will be referenced both in
+      // the SnapshotManager as well as in the file system tree. The snapshot
+      // count will go up to 1.
+      Path snapshotPath = fs.createSnapshot(TEST_PATH);
+      assertEquals(1, nnSnapshotManager.getNumSnapshots());
+      assertEquals(1, nnSnapshotManager.getNumSnapshottableDirs());
+      
+      // 3. Start up a 2NN and have it do a checkpoint. It will have foo and its
+      // snapshot in its list of snapshottable dirs referenced from the
+      // SnapshotManager, as well as in the file system tree.
+      secondary.doCheckpoint();
+      assertEquals(1, secondarySnapshotManager.getNumSnapshots());
+      assertEquals(1, secondarySnapshotManager.getNumSnapshottableDirs());
+      
+      // 4. Disallow snapshots on and delete foo on the NN. The snapshot count
+      // will go down to 0 and the snapshottable dir will be removed from the fs
+      // tree.
+      fs.deleteSnapshot(TEST_PATH, snapshotPath.getName());
+      admin.disallowSnapshot(TEST_PATH);
+      assertEquals(0, nnSnapshotManager.getNumSnapshots());
+      assertEquals(0, nnSnapshotManager.getNumSnapshottableDirs());
+      
+      // 5. Have the NN do a saveNamespace, writing out a new fsimage with
+      // snapshot count 0.
+      NameNodeAdapter.enterSafeMode(cluster.getNameNode(), false);
+      NameNodeAdapter.saveNamespace(cluster.getNameNode());
+      NameNodeAdapter.leaveSafeMode(cluster.getNameNode());
+      
+      // 6. Have the still-running 2NN do a checkpoint. It will notice that the
+      // fsimage has changed on the NN and redownload/reload from that image.
+      // This will replace all INodes in the file system tree as well as reset
+      // the snapshot counter to 0 in the SnapshotManager. However, it will not
+      // clear the list of snapshottable dirs referenced from the
+      // SnapshotManager. When it writes out an fsimage, the 2NN will write out
+      // 0 for the snapshot count, but still serialize the snapshottable dir
+      // referenced in the SnapshotManager even though it no longer appears in
+      // the file system tree. The NN will not be able to start up with this.
+      secondary.doCheckpoint();
+      assertEquals(0, secondarySnapshotManager.getNumSnapshots());
+      assertEquals(0, secondarySnapshotManager.getNumSnapshottableDirs());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (secondary != null) {
+        secondary.shutdown();
+      }
+    }
+
+  }
+
+}

+ 18 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -34,6 +34,7 @@ import java.net.HttpURLConnection;
 import java.net.URL;
 
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.junit.Test;
 
 public class TestByteRangeInputStream {
@@ -41,24 +42,24 @@ public static class MockHttpURLConnection extends HttpURLConnection {
   public MockHttpURLConnection(URL u) {
     super(u);
   }
-  
+
   @Override
   public boolean usingProxy(){
     return false;
   }
-  
+
   @Override
   public void disconnect() {
   }
-  
+
   @Override
   public void connect() {
   }
-  
+
   @Override
   public InputStream getInputStream() throws IOException {
     return new ByteArrayInputStream("asdf".getBytes());
-  } 
+  }
 
   @Override
   public URL getURL() {
@@ -70,7 +71,7 @@ public static class MockHttpURLConnection extends HttpURLConnection {
     }
     return u;
   }
-  
+
   @Override
   public int getResponseCode() {
     if (responseCode != -1) {
@@ -87,13 +88,13 @@ public static class MockHttpURLConnection extends HttpURLConnection {
   public void setResponseCode(int resCode) {
     responseCode = resCode;
   }
-  
+
   @Override
   public String getHeaderField(String field) {
     return (field.equalsIgnoreCase(StreamFile.CONTENT_LENGTH)) ? "65535" : null;
   }
 }
-  
+
   @Test
   public void testByteRange() throws IOException {
     HftpFileSystem.RangeHeaderUrlOpener ospy = spy(
@@ -149,7 +150,7 @@ public static class MockHttpURLConnection extends HttpURLConnection {
 
     ((MockHttpURLConnection) rspy.openConnection()).setResponseCode(200);
     is.seek(500);
-    
+
     try {
       is.read();
       fail("Exception should be thrown when 200 response is given "
@@ -171,31 +172,31 @@ public static class MockHttpURLConnection extends HttpURLConnection {
                    "HTTP_OK expected, received 206", e.getMessage());
     }
   }
-  
+
   @Test
   public void testPropagatedClose() throws IOException {
     ByteRangeInputStream brs = spy(
         new HftpFileSystem.RangeHeaderInputStream(new URL("http://test/")));
-    
+
     InputStream mockStream = mock(InputStream.class);
     doReturn(mockStream).when(brs).openInputStream();
 
     int brisOpens = 0;
     int brisCloses = 0;
     int isCloses = 0;
-    
+
     // first open, shouldn't close underlying stream
     brs.getInputStream();
     verify(brs, times(++brisOpens)).openInputStream();
     verify(brs, times(brisCloses)).close();
     verify(mockStream, times(isCloses)).close();
-    
+
     // stream is open, shouldn't close underlying stream
     brs.getInputStream();
     verify(brs, times(brisOpens)).openInputStream();
     verify(brs, times(brisCloses)).close();
     verify(mockStream, times(isCloses)).close();
-    
+
     // seek forces a reopen, should close underlying stream
     brs.seek(1);
     brs.getInputStream();
@@ -221,12 +222,12 @@ public static class MockHttpURLConnection extends HttpURLConnection {
     brs.close();
     verify(brs, times(++brisCloses)).close();
     verify(mockStream, times(++isCloses)).close();
-    
+
     // it's already closed, underlying stream should not close
     brs.close();
     verify(brs, times(++brisCloses)).close();
     verify(mockStream, times(isCloses)).close();
-    
+
     // it's closed, don't reopen it
     boolean errored = false;
     try {

+ 24 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpDelegationToken.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpDelegationToken.java

@@ -16,10 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.net.ServerSocket;
@@ -29,7 +30,10 @@ import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
+import org.apache.hadoop.hdfs.web.HsftpFileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -46,11 +50,11 @@ public class TestHftpDelegationToken {
     final Configuration conf = new Configuration();
     conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
     UserGroupInformation.setConfiguration(conf);
-    UserGroupInformation user =  
-      UserGroupInformation.createUserForTesting("oom", 
+    UserGroupInformation user =
+      UserGroupInformation.createUserForTesting("oom",
                                                 new String[]{"memory"});
     Token<?> token = new Token<TokenIdentifier>
-      (new byte[0], new byte[0], 
+      (new byte[0], new byte[0],
        DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
        new Text("127.0.0.1:8020"));
     user.addToken(token);
@@ -58,7 +62,7 @@ public class TestHftpDelegationToken {
       (null, null, new Text("other token"), new Text("127.0.0.1:8021"));
     user.addToken(token2);
     assertEquals("wrong tokens in user", 2, user.getTokens().size());
-    FileSystem fs = 
+    FileSystem fs =
       user.doAs(new PrivilegedExceptionAction<FileSystem>() {
 	  @Override
     public FileSystem run() throws Exception {
@@ -78,13 +82,13 @@ public class TestHftpDelegationToken {
 
     Configuration conf = new Configuration();
     conf.setClass("fs.hftp.impl", MyHftpFileSystem.class, FileSystem.class);
-    
+
     int httpPort = 80;
     int httpsPort = 443;
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, httpPort);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, httpsPort);
-    
-    // test with implicit default port 
+
+    // test with implicit default port
     URI fsUri = URI.create("hftp://localhost");
     MyHftpFileSystem fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
@@ -96,14 +100,14 @@ public class TestHftpDelegationToken {
     fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpPort, conf);
-    
+
     // test with non-default port
     // Make sure it uses the port from the hftp URI.
     fsUri = URI.create("hftp://localhost:"+(httpPort+1));
     fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpPort + 1, conf);
-    
+
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 5);
   }
 
@@ -119,27 +123,27 @@ public class TestHftpDelegationToken {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, httpPort);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, httpsPort);
 
-    // test with implicit default port 
+    // test with implicit default port
     URI fsUri = URI.create("hsftp://localhost");
     MyHsftpFileSystem fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpsPort, fs.getCanonicalUri().getPort());
-    checkTokenSelection(fs, httpsPort, conf); 
+    checkTokenSelection(fs, httpsPort, conf);
 
     // test with explicit default port
     fsUri = URI.create("hsftp://localhost:"+httpsPort);
     fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpsPort, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpsPort, conf);
-    
+
     // test with non-default port
     fsUri = URI.create("hsftp://localhost:"+(httpsPort+1));
     fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpsPort+1, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpsPort+1, conf);
-    
+
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 5);
   }
-  
+
 
   @Test
   public void testInsecureRemoteCluster()  throws Exception {
@@ -186,7 +190,7 @@ public class TestHftpDelegationToken {
       t.interrupt();
     }
   }
-  
+
   private void checkTokenSelection(HftpFileSystem fs,
                                    int port,
                                    Configuration conf) throws IOException {
@@ -216,12 +220,12 @@ public class TestHftpDelegationToken {
     token = fs.selectDelegationToken(ugi);
     assertNotNull(token);
     assertEquals(hftpToken, token);
-    
+
     // switch to using host-based tokens, no token should match
     SecurityUtilTestHelper.setTokenServiceUseIp(false);
     token = fs.selectDelegationToken(ugi);
     assertNull(token);
-    
+
     // test fallback to hdfs token
     hdfsToken = new Token<TokenIdentifier>(
         new byte[0], new byte[0],
@@ -241,7 +245,7 @@ public class TestHftpDelegationToken {
     assertNotNull(token);
     assertEquals(hftpToken, token);
   }
-  
+
   static class MyHftpFileSystem extends HftpFileSystem {
     @Override
     public URI getCanonicalUri() {
@@ -255,7 +259,7 @@ public class TestHftpDelegationToken {
     @Override
     protected void initDelegationToken() throws IOException {}
   }
-  
+
   static class MyHsftpFileSystem extends HsftpFileSystem {
     @Override
     public URI getCanonicalUri() {

+ 30 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpFileSystem.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -38,16 +38,21 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster.Builder;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
+import org.apache.hadoop.hdfs.web.HsftpFileSystem;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.log4j.Level;
 import org.junit.*;
 
 public class TestHftpFileSystem {
   private static final Random RAN = new Random();
-  
+
   private static Configuration config = null;
   private static MiniDFSCluster cluster = null;
   private static String blockPoolId = null;
@@ -94,17 +99,17 @@ public class TestHftpFileSystem {
     config = new Configuration();
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(2).build();
     blockPoolId = cluster.getNamesystem().getBlockPoolId();
-    hftpUri = 
+    hftpUri =
       "hftp://" + config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
   }
-  
+
   @AfterClass
   public static void tearDown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
-  
+
   @Before
   public void initFileSystems() throws IOException {
     hdfs = cluster.getFileSystem();
@@ -119,9 +124,9 @@ public class TestHftpFileSystem {
   public void resetFileSystems() throws IOException {
     FileSystem.closeAll();
   }
-  
+
   /**
-   * Test file creation and access with file names that need encoding. 
+   * Test file creation and access with file names that need encoding.
    */
   @Test
   public void testFileNameEncoding() throws IOException, URISyntaxException {
@@ -159,13 +164,13 @@ public class TestHftpFileSystem {
 
     // Get the path's block location so we can determine
     // if we were redirected to the right DN.
-    BlockLocation[] locations = 
+    BlockLocation[] locations =
         hdfs.getFileBlockLocations(path, 0, 10);
     String xferAddr = locations[0].getNames()[0];
 
     // Connect to the NN to get redirected
     URL u = hftpFs.getNamenodeURL(
-        "/data" + ServletUtil.encodePath(path.toUri().getPath()), 
+        "/data" + ServletUtil.encodePath(path.toUri().getPath()),
         "ugi=userx,groupy");
     HttpURLConnection conn = (HttpURLConnection)u.openConnection();
     HttpURLConnection.setFollowRedirects(true);
@@ -176,7 +181,7 @@ public class TestHftpFileSystem {
     // Find the datanode that has the block according to locations
     // and check that the URL was redirected to this DN's info port
     for (DataNode node : cluster.getDataNodes()) {
-      DatanodeRegistration dnR = 
+      DatanodeRegistration dnR =
         DataNodeTestUtils.getDNRegistrationForBP(node, blockPoolId);
       if (dnR.getXferAddr().equals(xferAddr)) {
         checked = true;
@@ -207,25 +212,25 @@ public class TestHftpFileSystem {
     FSDataOutputStream out = hdfs.create(testFile, true);
     out.writeBytes("0123456789");
     out.close();
-    
+
     FSDataInputStream in = hftpFs.open(testFile);
-    
+
     // Test read().
     for (int i = 0; i < 5; ++i) {
       assertEquals(i, in.getPos());
       in.read();
     }
-    
+
     // Test read(b, off, len).
     assertEquals(5, in.getPos());
     byte[] buffer = new byte[10];
     assertEquals(2, in.read(buffer, 0, 2));
     assertEquals(7, in.getPos());
-    
+
     // Test read(b).
     int bytesRead = in.read(buffer);
     assertEquals(7 + bytesRead, in.getPos());
-    
+
     // Test EOF.
     for (int i = 0; i < 100; ++i) {
       in.read();
@@ -261,21 +266,21 @@ public class TestHftpFileSystem {
     in.close();
     checkClosedStream(in);
     checkClosedStream(in.getWrappedStream());
-    
+
     // force the stream to connect and then close it
     in = hftpFs.open(testFile);
-    int ch = in.read(); 
+    int ch = in.read();
     assertEquals('0', ch);
     in.close();
     checkClosedStream(in);
     checkClosedStream(in.getWrappedStream());
-    
+
     // make sure seeking doesn't automagically reopen the stream
     in.seek(4);
     checkClosedStream(in);
     checkClosedStream(in.getWrappedStream());
   }
-  
+
   private void checkClosedStream(InputStream is) {
     IOException ioe = null;
     try {
@@ -286,7 +291,7 @@ public class TestHftpFileSystem {
     assertNotNull("No exception on closed read", ioe);
     assertEquals("Stream closed", ioe.getMessage());
   }
-  
+
   @Test
   public void testHftpDefaultPorts() throws IOException {
     Configuration conf = new Configuration();
@@ -304,7 +309,7 @@ public class TestHftpFileSystem {
         fs.getCanonicalServiceName()
     );
   }
-  
+
   @Test
   public void testHftpCustomDefaultPorts() throws IOException {
     Configuration conf = new Configuration();
@@ -314,7 +319,7 @@ public class TestHftpFileSystem {
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
     assertEquals(123, fs.getDefaultPort());
-    
+
     assertEquals(uri, fs.getUri());
 
     // HFTP uses http to get the token so canonical service name should
@@ -349,8 +354,8 @@ public class TestHftpFileSystem {
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
     assertEquals(123, fs.getDefaultPort());
-   
-    assertEquals(uri, fs.getUri()); 
+
+    assertEquals(uri, fs.getUri());
     assertEquals(
         "127.0.0.1:789",
         fs.getCanonicalServiceName()
@@ -384,7 +389,7 @@ public class TestHftpFileSystem {
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
     assertEquals(456, fs.getDefaultPort());
-    
+
     assertEquals(uri, fs.getUri());
     assertEquals(
         "127.0.0.1:456",

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpURLTimeouts.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.web;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -33,6 +33,8 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
+import org.apache.hadoop.hdfs.web.HsftpFileSystem;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.junit.Test;
 
@@ -83,7 +85,7 @@ public class TestHftpURLTimeouts {
 
     HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf);
     fs.connectionFactory = new URLConnectionFactory(5);
-    
+
     try {
       HttpURLConnection conn = null;
       timedout = false;
@@ -104,7 +106,7 @@ public class TestHftpURLTimeouts {
       fs.close();
     }
   }
-  
+
   private boolean checkConnectTimeout(HftpFileSystem fs, boolean ignoreReadTimeout)
       throws IOException {
     boolean timedout = false;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java

@@ -34,9 +34,9 @@ import java.util.concurrent.Executors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
+import org.apache.hadoop.hdfs.web.HftpFileSystem;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java

@@ -72,7 +72,7 @@ public class TestCombineTextInputFormat {
     new Path(new Path(System.getProperty("test.build.data", "."), "data"),
              "TestCombineTextInputFormat");
 
-  @Test//(timeout=10000)
+  @Test(timeout=10000)
   public void testFormat() throws Exception {
     Job job = Job.getInstance(new Configuration(defaultConf));
 

+ 1 - 0
hadoop-project/src/site/site.xml

@@ -80,6 +80,7 @@
       <item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
       <item name="Short Circuit Local Reads" 
           href="hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html"/>
+      <item name="Centralized Cache Management" href="hadoop-project-dist/hadoop-hdfs/CentralizedCacheManagement.html"/>
       <item name="HDFS NFS Gateway" href="hadoop-project-dist/hadoop-hdfs/HdfsNfsGateway.html"/>
     </menu>
 

+ 0 - 1
hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCpV1.java

@@ -46,7 +46,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;

+ 0 - 7
hadoop-tools/hadoop-sls/src/main/sample-conf/capacity-scheduler.xml

@@ -57,11 +57,4 @@
     <name>yarn.scheduler.capacity.root.sls_queue_3.maximum-capacity</name>
     <value>100</value>
   </property>
-  
-  <property>
-    <name>yarn.scheduler.capacity.maximum-applications</name>
-    <value>1000</value>
-    <description>Maximum number of applications in the system which 
-    can be concurrently active both running and pending</description>
-  </property>
 </configuration>

+ 0 - 50
hadoop-tools/hadoop-sls/src/main/sample-conf/fair-scheduler-allocation.xml

@@ -1,50 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<!--
-  This file contains pool and user allocations for the Fair Scheduler.
-  Its format is explained in the Fair Scheduler documentation at
-  http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.html
-  The documentation also includes a sample config file.
--->
-
-<allocations>
-  <user name="jenkins">
-    <!-- Limit on running jobs for the user across all pools. If more
-      jobs than this are submitted, only the first <maxRunningJobs> will
-      be scheduled at any given time. Defaults to infinity or the
-      userMaxJobsDefault value set below. -->
-    <maxRunningJobs>1000</maxRunningJobs>
-  </user>
-  <userMaxAppsDefault>1000</userMaxAppsDefault>
-  <queue name="sls_queue_1">
-    <minResources>1024 mb, 1 vcores</minResources>
-    <schedulingMode>fair</schedulingMode>
-    <weight>0.25</weight>
-    <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
-  </queue>
-  <queue name="sls_queue_2">
-    <minResources>1024 mb, 1 vcores</minResources>
-    <schedulingMode>fair</schedulingMode>
-    <weight>0.25</weight>
-    <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
-  </queue>
-  <queue name="sls_queue_3">
-    <minResources>1024 mb, 1 vcores</minResources>
-    <weight>0.5</weight>
-    <schedulingMode>fair</schedulingMode>
-    <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
-  </queue>
-</allocations>

+ 28 - 25
hadoop-tools/hadoop-sls/src/main/sample-conf/fair-scheduler.xml

@@ -20,28 +20,31 @@
   The documentation also includes a sample config file.
 -->
 
-<configuration>
-  <property>
-    <description>Absolute path to allocation file. An allocation file is an XML
-    manifest describing queues and their properties, in addition to certain
-    policy defaults. This file must be in XML format as described in
-    http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.html.
-    </description>
-    <name>yarn.scheduler.fair.allocation.file</name>
-    <value>fair-scheduler-allocation.xml</value>
-  </property>
-
-  <property>
-    <description>Whether to use preemption. Note that preemption is experimental
-    in the current version. Defaults to false.</description>
-    <name>yarn.scheduler.fair.preemption</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <description>Whether to allow multiple container assignments in one
-    heartbeat. Defaults to false.</description>
-    <name>yarn.scheduler.fair.assignmultiple</name>
-    <value>true</value>
-  </property>
-</configuration>
+<allocations>
+  <user name="jenkins">
+    <!-- Limit on running jobs for the user across all pools. If more
+      jobs than this are submitted, only the first <maxRunningJobs> will
+      be scheduled at any given time. Defaults to infinity or the
+      userMaxJobsDefault value set below. -->
+    <maxRunningJobs>1000</maxRunningJobs>
+  </user>
+  <userMaxAppsDefault>1000</userMaxAppsDefault>
+  <queue name="sls_queue_1">
+    <minResources>1024 mb, 1 vcores</minResources>
+    <schedulingMode>fair</schedulingMode>
+    <weight>0.25</weight>
+    <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
+  </queue>
+  <queue name="sls_queue_2">
+    <minResources>1024 mb, 1 vcores</minResources>
+    <schedulingMode>fair</schedulingMode>
+    <weight>0.25</weight>
+    <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
+  </queue>
+  <queue name="sls_queue_3">
+    <minResources>1024 mb, 1 vcores</minResources>
+    <weight>0.5</weight>
+    <schedulingMode>fair</schedulingMode>
+    <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
+  </queue>
+</allocations>

+ 20 - 0
hadoop-tools/hadoop-sls/src/main/sample-conf/yarn-site.xml

@@ -57,4 +57,24 @@
     <value>false</value>
   </property>
 
+  <property>
+    <name>yarn.scheduler.capacity.maximum-applications</name>
+    <value>1000</value>
+    <description>Maximum number of applications in the system which
+      can be concurrently active both running and pending</description>
+  </property>
+
+  <property>
+    <description>Whether to use preemption. Note that preemption is experimental
+      in the current version. Defaults to false.</description>
+    <name>yarn.scheduler.fair.preemption</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>Whether to allow multiple container assignments in one
+      heartbeat. Defaults to false.</description>
+    <name>yarn.scheduler.fair.assignmultiple</name>
+    <value>true</value>
+  </property>
 </configuration>

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

@@ -70,6 +70,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9598. Improve code coverage of RMAdminCLI (Aleksey Gorshkov and
     Andrey Klochkov via jeagles)
 
+    YARN-1306. Clean up hadoop-sls sample-conf according to YARN-1228 (Wei Yan
+    via Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES