فهرست منبع

Merge r1569890 through r1570391 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1570392 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 سال پیش
والد
کامیت
37afb4d683
16فایلهای تغییر یافته به همراه229 افزوده شده و 140 حذف شده
  1. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 0 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
  3. 70 46
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
  4. 12 6
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  5. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  6. 3 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  7. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  8. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  9. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java
  10. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
  11. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  12. 0 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHttpPolicy.java
  13. 48 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  14. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
  15. 3 0
      hadoop-yarn-project/CHANGES.txt
  16. 10 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java

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

@@ -363,6 +363,9 @@ Release 2.4.0 - UNRELEASED
 
     HADOOP-10343. Change info to debug log in LossyRetryInvocationHandler. (arpit)
 
+    HADOOP-10348. Deprecate hadoop.ssl.configuration in branch-2, and remove
+    it in trunk. (Haohui Mai via jing9)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -390,6 +393,9 @@ Release 2.4.0 - UNRELEASED
 
     HADOOP-10346. Deadlock while logging tokens (jlowe)
 
+    HADOOP-10328. loadGenerator exit code is not reliable.
+    (Haohui Mai via cnauroth)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java

@@ -268,21 +268,11 @@ public class CommonConfigurationKeysPublic {
   public static final String  HADOOP_SECURITY_AUTH_TO_LOCAL =
     "hadoop.security.auth_to_local";
 
-  public static final String HADOOP_SSL_ENABLED_KEY = "hadoop.ssl.enabled";
-  public static final boolean HADOOP_SSL_ENABLED_DEFAULT = false;
-
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN =
           "hadoop.kerberos.min.seconds.before.relogin";
   /** Default value for HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN */
   public static final int HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT =
           60;
-
-  // HTTP policies to be used in configuration
-  // Use HttpPolicy.name() instead
-  @Deprecated
-  public static final String HTTP_POLICY_HTTP_ONLY = "HTTP_ONLY";
-  @Deprecated
-  public static final String HTTP_POLICY_HTTPS_ONLY = "HTTPS_ONLY";
 }
 

+ 70 - 46
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java

@@ -26,6 +26,7 @@ import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.Random;
 
@@ -39,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.CreateOpts;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -137,11 +139,15 @@ public class LoadGenerator extends Configured implements Tool {
     "-startTime <startTimeInMillis>\n" +
     "-scriptFile <filename>";
   final private String hostname;
-  
+  private final byte[] WRITE_CONTENTS = new byte[4096];
+
+  private static final int ERR_TEST_FAILED = 2;
+
   /** Constructor */
   public LoadGenerator() throws IOException, UnknownHostException {
     InetAddress addr = InetAddress.getLocalHost();
     hostname = addr.getHostName();
+    Arrays.fill(WRITE_CONTENTS, (byte) 'a');
   }
 
   private final static int OPEN = 0;
@@ -178,7 +184,8 @@ public class LoadGenerator extends Configured implements Tool {
     private long [] executionTime = new long[TOTAL_OP_TYPES];
     private long [] totalNumOfOps = new long[TOTAL_OP_TYPES];
     private byte[] buffer = new byte[1024];
-    
+    private boolean failed;
+
     private DFSClientThread(int id) {
       this.id = id;
     }
@@ -196,6 +203,7 @@ public class LoadGenerator extends Configured implements Tool {
       } catch (Exception ioe) {
         System.err.println(ioe.getLocalizedMessage());
         ioe.printStackTrace();
+        failed = true;
       }
     }
     
@@ -272,6 +280,35 @@ public class LoadGenerator extends Configured implements Tool {
       executionTime[LIST] += (Time.now()-startTime);
       totalNumOfOps[LIST]++;
     }
+
+    /** Create a file with a length of <code>fileSize</code>.
+     * The file is filled with 'a'.
+     */
+    private void genFile(Path file, long fileSize) throws IOException {
+      long startTime = Time.now();
+      FSDataOutputStream out = null;
+      try {
+        out = fc.create(file,
+            EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
+            CreateOpts.createParent(), CreateOpts.bufferSize(4096),
+            CreateOpts.repFac((short) 3));
+        executionTime[CREATE] += (Time.now() - startTime);
+        totalNumOfOps[CREATE]++;
+
+        long i = fileSize;
+        while (i > 0) {
+          long s = Math.min(fileSize, WRITE_CONTENTS.length);
+          out.write(WRITE_CONTENTS, 0, (int) s);
+          i -= s;
+        }
+
+        startTime = Time.now();
+        executionTime[WRITE_CLOSE] += (Time.now() - startTime);
+        totalNumOfOps[WRITE_CLOSE]++;
+      } finally {
+        IOUtils.cleanup(LOG, out);
+      }
+    }
   }
   
   /** Main function:
@@ -319,13 +356,21 @@ public class LoadGenerator extends Configured implements Tool {
     if(LOG.isDebugEnabled()) {
       LOG.debug("Done with testing.  Waiting for threads to finish.");
     }
+
+    boolean failed = false;
     for (DFSClientThread thread : threads) {
       thread.join();
       for (int i=0; i<TOTAL_OP_TYPES; i++) {
         executionTime[i] += thread.executionTime[i];
         totalNumOfOps[i] += thread.totalNumOfOps[i];
       }
+      failed = failed || thread.failed;
+    }
+
+    if (failed) {
+      exitCode = -ERR_TEST_FAILED;
     }
+
     long totalOps = 0;
     for (int i=0; i<TOTAL_OP_TYPES; i++) {
       totalOps += totalNumOfOps[i];
@@ -462,50 +507,50 @@ public class LoadGenerator extends Configured implements Tool {
     
     String line;
     // Read script, parse values, build array of duration, read and write probs
-    while((line = br.readLine()) != null) {
+
+    while ((line = br.readLine()) != null) {
       lineNum++;
-      if(line.startsWith("#") || line.isEmpty()) // skip comments and blanks
+      if (line.startsWith("#") || line.isEmpty()) // skip comments and blanks
         continue;
-      
+
       String[] a = line.split("\\s");
-      if(a.length != 3) {
-        System.err.println("Line " + lineNum + 
-                           ": Incorrect number of parameters: " + line);
+      if (a.length != 3) {
+        System.err.println("Line " + lineNum
+            + ": Incorrect number of parameters: " + line);
       }
-      
+
       try {
         long d = Long.parseLong(a[0]);
-        if(d < 0) { 
-           System.err.println("Line " + lineNum + ": Invalid duration: " + d);
-           return -1;
+        if (d < 0) {
+          System.err.println("Line " + lineNum + ": Invalid duration: " + d);
+          return -1;
         }
 
         double r = Double.parseDouble(a[1]);
-        if(r < 0.0 || r > 1.0 ) {
-           System.err.println("Line " + lineNum + 
-                      ": The read probability must be [0, 1]: " + r);
-           return -1;
+        if (r < 0.0 || r > 1.0) {
+          System.err.println("Line " + lineNum
+              + ": The read probability must be [0, 1]: " + r);
+          return -1;
         }
-        
+
         double w = Double.parseDouble(a[2]);
-        if(w < 0.0 || w > 1.0) {
-          System.err.println("Line " + lineNum + 
-                       ": The read probability must be [0, 1]: " + r);
+        if (w < 0.0 || w > 1.0) {
+          System.err.println("Line " + lineNum
+              + ": The read probability must be [0, 1]: " + r);
           return -1;
         }
-        
+
         readProb.add(r);
         duration.add(d);
         writeProb.add(w);
-      } catch( NumberFormatException nfe) {
+      } catch (NumberFormatException nfe) {
         System.err.println(lineNum + ": Can't parse: " + line);
         return -1;
+      } finally {
+        IOUtils.cleanup(LOG, br);
       }
     }
     
-    br.close();
-    fr.close();
-    
     // Copy vectors to arrays of values, to avoid autoboxing overhead later
     durations = new long[duration.size()];
     readProbs = new double[readProb.size()];
@@ -581,27 +626,6 @@ public class LoadGenerator extends Configured implements Tool {
       }
     }
   }
-
-  /** Create a file with a length of <code>fileSize</code>.
-   * The file is filled with 'a'.
-   */
-  private void genFile(Path file, long fileSize) throws IOException {
-    long startTime = Time.now();
-    FSDataOutputStream out = fc.create(file,
-        EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
-        CreateOpts.createParent(), CreateOpts.bufferSize(4096),
-        CreateOpts.repFac((short) 3));
-    executionTime[CREATE] += (Time.now()-startTime);
-    totalNumOfOps[CREATE]++;
-
-    for (long i=0; i<fileSize; i++) {
-      out.writeByte('a');
-    }
-    startTime = Time.now();
-    out.close();
-    executionTime[WRITE_CLOSE] += (Time.now()-startTime);
-    totalNumOfOps[WRITE_CLOSE]++;
-  }
   
   /** Main program
    * 

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

@@ -253,12 +253,6 @@ Trunk (Unreleased)
     HDFS-5719. FSImage#doRollback() should close prevState before return
     (Ted Yu via brandonli)
 
-    HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager 
-    (Haohui Mai via brandonli)
-
-    HDFS-5775. Consolidate the code for serialization in CacheManager
-    (Haohui Mai via brandonli)
-
     HDFS-5794. Fix the inconsistency of layout version number of 
     ADD_DATANODE_AND_STORAGE_UUIDS between trunk and branch-2. (jing9)
 
@@ -424,6 +418,12 @@ Release 2.4.0 - UNRELEASED
     HDFS-5868. Make hsync implementation pluggable on the DataNode.
     (Buddy Taylor via Arpit Agarwal)
 
+    HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager 
+    (Haohui Mai via brandonli)
+
+    HDFS-5775. Consolidate the code for serialization in CacheManager
+    (Haohui Mai via brandonli)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@@ -537,6 +537,12 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5979. Typo and logger fix for fsimage PB code. (wang)
 
+    HDFS-5962. Mtime and atime are not persisted for symbolic links. (Akira
+    Ajisaka via kihwal)
+
+    HDFS-5944. LeaseManager:findLeaseWithPrefixPath can't handle path like /a/b/
+    and cause SecondaryNameNode failed do checkpoint (Yunjiong Zhao via brandonli)
+
   BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
 
     HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)

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

@@ -377,8 +377,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT = 10;
   public static final String  DFS_SUPPORT_APPEND_KEY = "dfs.support.append";
   public static final boolean DFS_SUPPORT_APPEND_DEFAULT = true;
-  public static final String  DFS_HTTPS_ENABLE_KEY = "dfs.https.enable";
-  public static final boolean DFS_HTTPS_ENABLE_DEFAULT = false;
   public static final String  DFS_HTTP_POLICY_KEY = "dfs.http.policy";
   public static final String  DFS_HTTP_POLICY_DEFAULT =  HttpConfig.Policy.HTTP_ONLY.name();
   public static final String  DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY = "dfs.default.chunk.view.size";

+ 3 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -1553,44 +1553,11 @@ public class DFSUtil {
   }
 
   /**
-   * Get http policy. Http Policy is chosen as follows:
-   * <ol>
-   * <li>If hadoop.ssl.enabled is set, http endpoints are not started. Only
-   * https endpoints are started on configured https ports</li>
-   * <li>This configuration is overridden by dfs.https.enable configuration, if
-   * it is set to true. In that case, both http and https endpoints are stared.</li>
-   * <li>All the above configurations are overridden by dfs.http.policy
-   * configuration. With this configuration you can set http-only, https-only
-   * and http-and-https endpoints.</li>
-   * </ol>
-   * See hdfs-default.xml documentation for more details on each of the above
-   * configuration settings.
+   * Get http policy.
    */
   public static HttpConfig.Policy getHttpPolicy(Configuration conf) {
-    String policyStr = conf.get(DFSConfigKeys.DFS_HTTP_POLICY_KEY);
-    if (policyStr == null) {
-      boolean https = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY,
-          DFSConfigKeys.DFS_HTTPS_ENABLE_DEFAULT);
-
-      boolean hadoopSsl = conf.getBoolean(
-          CommonConfigurationKeys.HADOOP_SSL_ENABLED_KEY,
-          CommonConfigurationKeys.HADOOP_SSL_ENABLED_DEFAULT);
-
-      if (hadoopSsl) {
-        LOG.warn(CommonConfigurationKeys.HADOOP_SSL_ENABLED_KEY
-            + " is deprecated. Please use " + DFSConfigKeys.DFS_HTTP_POLICY_KEY
-            + ".");
-      }
-      if (https) {
-        LOG.warn(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY
-            + " is deprecated. Please use " + DFSConfigKeys.DFS_HTTP_POLICY_KEY
-            + ".");
-      }
-
-      return (hadoopSsl || https) ? HttpConfig.Policy.HTTP_AND_HTTPS
-          : HttpConfig.Policy.HTTP_ONLY;
-    }
-
+    String policyStr = conf.get(DFSConfigKeys.DFS_HTTP_POLICY_KEY,
+        DFSConfigKeys.DFS_HTTP_POLICY_DEFAULT);
     HttpConfig.Policy policy = HttpConfig.Policy.fromString(policyStr);
     if (policy == null) {
       throw new HadoopIllegalArgumentException("Unregonized value '"

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -279,7 +279,8 @@ public final class FSImageFormatPBINode {
           parent.getLoaderContext().getStringTable());
 
       INodeSymlink sym = new INodeSymlink(n.getId(), n.getName().toByteArray(),
-          permissions, 0, 0, s.getTarget().toStringUtf8());
+          permissions, s.getModificationTime(), s.getAccessTime(),
+          s.getTarget().toStringUtf8());
 
       return sym;
     }
@@ -482,7 +483,9 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeSymlink.Builder b = INodeSection.INodeSymlink
           .newBuilder()
           .setPermission(buildPermissionStatus(n, state.getStringMap()))
-          .setTarget(ByteString.copyFrom(n.getSymlink()));
+          .setTarget(ByteString.copyFrom(n.getSymlink()))
+          .setModificationTime(n.getModificationTime())
+          .setAccessTime(n.getAccessTime());
 
       INodeSection.INode r = buildINodeCommon(n)
           .setType(INodeSection.INode.Type.SYMLINK).setSymlink(b).build();

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

@@ -339,7 +339,12 @@ public class LeaseManager {
     }
 
     final Map<String, Lease> entries = new HashMap<String, Lease>();
-    final int srclen = prefix.length();
+    int srclen = prefix.length();
+    
+    // prefix may ended with '/'
+    if (prefix.charAt(srclen - 1) == Path.SEPARATOR_CHAR) {
+      srclen -= 1;
+    }
 
     for(Map.Entry<String, Lease> entry : path2lease.tailMap(prefix).entrySet()) {
       final String p = entry.getKey();

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java

@@ -170,8 +170,9 @@ final class LsrPBImage {
       PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
           d.getPermission(), stringTable);
       out.print(String.format("-%s  - %8s %10s %10s %10d %s%s -> %s\n", p
-          .getPermission().toString(), p.getUserName(), p.getGroupName(), 0, 0,
-          parent, inode.getName().toStringUtf8(), d.getTarget().toStringUtf8()));
+          .getPermission().toString(), p.getUserName(), p.getGroupName(), d
+          .getModificationTime(), 0, parent, inode.getName().toStringUtf8(),
+          d.getTarget().toStringUtf8()));
     }
       break;
     default:

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java

@@ -289,8 +289,9 @@ public final class PBImageXmlWriter {
   }
 
   private void dumpINodeSymlink(INodeSymlink s) {
-    o("permission", dumpPermission(s.getPermission())).o("target",
-        s.getTarget().toStringUtf8());
+    o("permission", dumpPermission(s.getPermission()))
+        .o("target", s.getTarget().toStringUtf8())
+        .o("mtime", s.getModificationTime()).o("atime", s.getAccessTime());
   }
 
   private void dumpNameSection(InputStream in) throws IOException {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -131,6 +131,8 @@ message INodeSection {
   message INodeSymlink {
     optional fixed64 permission = 1;
     optional bytes target = 2;
+    optional uint64 modificationTime = 3;
+    optional uint64 accessTime = 4;
   }
 
   message INode {
@@ -301,4 +303,4 @@ message CacheManagerSection {
   required uint32 numDirectives   = 3;
   // repeated CachePoolInfoProto pools
   // repeated CacheDirectiveInfoProto directives
-}
+}

+ 0 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHttpPolicy.java

@@ -17,12 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.http.HttpConfig.Policy.HTTP_AND_HTTPS;
-import static org.apache.hadoop.http.HttpConfig.Policy.HTTP_ONLY;
-
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Assert;
 import org.junit.Test;
 
 public final class TestHttpPolicy {
@@ -33,22 +29,4 @@ public final class TestHttpPolicy {
     conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, "invalid");
     DFSUtil.getHttpPolicy(conf);
   }
-
-  @Test
-  public void testDeprecatedConfiguration() {
-    Configuration conf = new Configuration(false);
-    Assert.assertSame(HTTP_ONLY, DFSUtil.getHttpPolicy(conf));
-
-    conf.setBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, true);
-    Assert.assertSame(HTTP_AND_HTTPS, DFSUtil.getHttpPolicy(conf));
-
-    conf = new Configuration(false);
-    conf.setBoolean(DFSConfigKeys.HADOOP_SSL_ENABLED_KEY, true);
-    Assert.assertSame(HTTP_AND_HTTPS, DFSUtil.getHttpPolicy(conf));
-
-    conf = new Configuration(false);
-    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HTTP_ONLY.name());
-    conf.setBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, true);
-    Assert.assertSame(HTTP_ONLY, DFSUtil.getHttpPolicy(conf));
-  }
 }

+ 48 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -135,4 +136,51 @@ public class TestFSImage {
       }
     }
   }
+
+  /**
+   * Ensure mtime and atime can be loaded from fsimage.
+   */
+  @Test(timeout=60000)
+  public void testLoadMtimeAtime() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      DistributedFileSystem hdfs = cluster.getFileSystem();
+      String userDir = hdfs.getHomeDirectory().toUri().getPath().toString();
+      Path file = new Path(userDir, "file");
+      Path dir = new Path(userDir, "/dir");
+      Path link = new Path(userDir, "/link");
+      hdfs.createNewFile(file);
+      hdfs.mkdirs(dir);
+      hdfs.createSymlink(file, link, false);
+
+      long mtimeFile = hdfs.getFileStatus(file).getModificationTime();
+      long atimeFile = hdfs.getFileStatus(file).getAccessTime();
+      long mtimeDir = hdfs.getFileStatus(dir).getModificationTime();
+      long mtimeLink = hdfs.getFileLinkStatus(link).getModificationTime();
+      long atimeLink = hdfs.getFileLinkStatus(link).getAccessTime();
+
+      // save namespace and restart cluster
+      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+      hdfs.saveNamespace();
+      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+      cluster.shutdown();
+      cluster = new MiniDFSCluster.Builder(conf).format(false)
+          .numDataNodes(1).build();
+      cluster.waitActive();
+      hdfs = cluster.getFileSystem();
+      
+      assertEquals(mtimeFile, hdfs.getFileStatus(file).getModificationTime());
+      assertEquals(atimeFile, hdfs.getFileStatus(file).getAccessTime());
+      assertEquals(mtimeDir, hdfs.getFileStatus(dir).getModificationTime());
+      assertEquals(mtimeLink, hdfs.getFileLinkStatus(link).getModificationTime());
+      assertEquals(atimeLink, hdfs.getFileLinkStatus(link).getAccessTime());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java

@@ -0,0 +1,57 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+
+public class TestLeaseManager {
+  Configuration conf = new HdfsConfiguration();
+  
+  @Test
+  public void testRemoveLeaseWithPrefixPath() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    cluster.waitActive();
+
+    LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem());
+    lm.addLease("holder1", "/a/b");
+    lm.addLease("holder2", "/a/c");
+    assertNotNull(lm.getLeaseByPath("/a/b"));
+    assertNotNull(lm.getLeaseByPath("/a/c"));
+
+    lm.removeLeaseWithPrefixPath("/a");
+
+    assertNull(lm.getLeaseByPath("/a/b"));
+    assertNull(lm.getLeaseByPath("/a/c"));
+
+    lm.addLease("holder1", "/a/b");
+    lm.addLease("holder2", "/a/c");
+
+    lm.removeLeaseWithPrefixPath("/a/");
+
+    assertNull(lm.getLeaseByPath("/a/b"));
+    assertNull(lm.getLeaseByPath("/a/c"));
+  }
+}

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

@@ -25,6 +25,9 @@ Release 2.5.0 - UNRELEASED
     YARN-1479. Invalid NaN values in Hadoop REST API JSON response (Chen He via
     jeagles)
 
+    YARN-1736. FS: AppSchedulable.assignContainer's priority argument is 
+    redundant. (Naren Koneru via kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES 

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

@@ -203,7 +203,7 @@ public class AppSchedulable extends Schedulable {
    * sure the particular request should be facilitated by this node.
    */
   private Resource assignContainer(FSSchedulerNode node,
-      Priority priority, ResourceRequest request, NodeType type,
+      ResourceRequest request, NodeType type,
       boolean reserved) {
 
     // How much does this request need?
@@ -216,25 +216,25 @@ public class AppSchedulable extends Schedulable {
     if (reserved) {
       container = node.getReservedContainer().getContainer();
     } else {
-      container = createContainer(app, node, capability, priority);
+      container = createContainer(app, node, capability, request.getPriority());
     }
 
     // Can we allocate a container on this node?
     if (Resources.fitsIn(capability, available)) {
       // Inform the application of the new container for this request
       RMContainer allocatedContainer =
-          app.allocate(type, node, priority, request, container);
+          app.allocate(type, node, request.getPriority(), request, container);
       if (allocatedContainer == null) {
         // Did the application need this resource?
         if (reserved) {
-          unreserve(priority, node);
+          unreserve(request.getPriority(), node);
         }
         return Resources.none();
       }
 
       // If we had previously made a reservation, delete it
       if (reserved) {
-        unreserve(priority, node);
+        unreserve(request.getPriority(), node);
       }
 
       // Inform the node
@@ -244,7 +244,7 @@ public class AppSchedulable extends Schedulable {
       return container.getResource();
     } else {
       // The desired container won't fit here, so reserve
-      reserve(priority, node, container, reserved);
+      reserve(request.getPriority(), node, container, reserved);
 
       return FairScheduler.CONTAINER_RESERVED;
     }
@@ -307,8 +307,8 @@ public class AppSchedulable extends Schedulable {
 
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && localRequest != null && localRequest.getNumContainers() != 0) {
-          return assignContainer(node, priority,
-              localRequest, NodeType.NODE_LOCAL, reserved);
+          return assignContainer(node, localRequest,
+              NodeType.NODE_LOCAL, reserved);
         }
         
         if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) {
@@ -318,7 +318,7 @@ public class AppSchedulable extends Schedulable {
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
                 allowedLocality.equals(NodeType.OFF_SWITCH))) {
-          return assignContainer(node, priority, rackLocalRequest,
+          return assignContainer(node, rackLocalRequest,
               NodeType.RACK_LOCAL, reserved);
         }
 
@@ -330,7 +330,7 @@ public class AppSchedulable extends Schedulable {
         
         if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0
             && allowedLocality.equals(NodeType.OFF_SWITCH)) {
-          return assignContainer(node, priority, offSwitchRequest,
+          return assignContainer(node, offSwitchRequest,
               NodeType.OFF_SWITCH, reserved);
         }
       }