浏览代码

Merge trunk into HDFS-6584

Jing Zhao 10 年之前
父节点
当前提交
8de20a1987
共有 16 个文件被更改,包括 447 次插入19 次删除
  1. 8 5
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 5 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java
  3. 12 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  4. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  5. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  7. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/BestEffortLongFile.java
  8. 18 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  9. 187 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  10. 121 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java
  11. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java
  12. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  13. 6 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  14. 3 0
      hadoop-yarn-project/CHANGES.txt
  15. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
  16. 48 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

+ 8 - 5
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -102,9 +102,6 @@ Trunk (Unreleased)
     HADOOP-9486. Promoted Windows and Shell related utils from YARN to Hadoop
     Common. (Chris Nauroth via vinodkv)
 
-    HADOOP-9540. Expose the InMemoryS3 and S3N FilesystemStores implementations
-    for Unit testing. (Hari via stevel)
-
     HADOOP-8844. Add a plaintext fs -text test-case.
     (Akira AJISAKA via harsh)
 
@@ -129,8 +126,6 @@ Trunk (Unreleased)
 
     HADOOP-11041. VersionInfo specifies subversion (Tsuyoshi OZAWA via aw)
 
-    HADOOP-10373 create tools/hadoop-amazon for aws/EMR support (stevel)
-
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -511,6 +506,11 @@ Release 2.6.0 - UNRELEASED
 
     HADOOP-10758. KMS: add ACLs on per key basis. (tucu)
 
+    HADOOP-9540. Expose the InMemoryS3 and S3N FilesystemStores implementations
+    for Unit testing. (Hari via stevel)
+
+    HADOOP-10373 create tools/hadoop-amazon for aws/EMR support (stevel)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
@@ -784,6 +784,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-9989. Bug introduced in HADOOP-9374, which parses the -tokenCacheFile 
     as binary file but set it to the configuration as JSON file. (zxu via tucu)
 
+    HADOOP-11085. Excessive logging by org.apache.hadoop.util.Progress when
+    value is NaN (Mit Desai via jlowe)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 5 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java

@@ -162,27 +162,27 @@ public class Progress {
   public synchronized void set(float progress) {
     if (Float.isNaN(progress)) {
       progress = 0;
-      LOG.warn("Illegal progress value found, progress is Float.NaN. " +
+      LOG.debug("Illegal progress value found, progress is Float.NaN. " +
         "Progress will be changed to 0");
     }
     else if (progress == Float.NEGATIVE_INFINITY) {
       progress = 0;
-      LOG.warn("Illegal progress value found, progress is " +
+      LOG.debug("Illegal progress value found, progress is " +
         "Float.NEGATIVE_INFINITY. Progress will be changed to 0");
     }
     else if (progress < 0) {
       progress = 0;
-      LOG.warn("Illegal progress value found, progress is less than 0." +
+      LOG.debug("Illegal progress value found, progress is less than 0." +
         " Progress will be changed to 0");
     }
     else if (progress > 1) {
       progress = 1;
-      LOG.warn("Illegal progress value found, progress is larger than 1." +
+      LOG.debug("Illegal progress value found, progress is larger than 1." +
         " Progress will be changed to 1");
     }
     else if (progress == Float.POSITIVE_INFINITY) {
       progress = 1;
-      LOG.warn("Illegal progress value found, progress is " +
+      LOG.debug("Illegal progress value found, progress is " +
         "Float.POSITIVE_INFINITY. Progress will be changed to 1");
     }
     this.progress = progress;

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

@@ -658,6 +658,15 @@ Release 2.6.0 - UNRELEASED
     HDFS-6506. Newly moved block replica been invalidated and deleted in
     TestBalancer. (Binglin Chang via cnauroth)
 
+    HDFS-6966. Add additional unit tests for encryption zones.
+    (Stephen Chu via wang)
+
+    HDFS-6621. Hadoop Balancer prematurely exits iterations.
+    (Rafal Wojdyla and Benjamin Bowman via wang)
+
+    HDFS-7045. Fix NameNode deadlock when opening file under /.reserved path.
+    (Yi Liu via wang)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an
@@ -765,6 +774,9 @@ Release 2.6.0 - UNRELEASED
     HDFS-6776. Using distcp to copy data between insecure and secure cluster via webdhfs 
     doesn't work. (yzhangal via tucu)
 
+    HDFS-7042. Upgrade fails for Windows HA cluster due to file locks held during
+    rename in JournalNode. (cnauroth)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java

@@ -988,6 +988,9 @@ public class Journal implements Closeable {
   }
 
   public synchronized void doPreUpgrade() throws IOException {
+    // Do not hold file lock on committedTxnId, because the containing
+    // directory will be renamed.  It will be reopened lazily on next access.
+    committedTxnId.close();
     storage.getJournalManager().doPreUpgrade();
   }
 
@@ -1037,7 +1040,10 @@ public class Journal implements Closeable {
         targetLayoutVersion);
   }
 
-  public void doRollback() throws IOException {
+  public synchronized void doRollback() throws IOException {
+    // Do not hold file lock on committedTxnId, because the containing
+    // directory will be renamed.  It will be reopened lazily on next access.
+    committedTxnId.close();
     storage.getJournalManager().doRollback();
   }
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java

@@ -725,6 +725,8 @@ public class Dispatcher {
           && (!srcBlocks.isEmpty() || blocksToReceive > 0)) {
         final PendingMove p = chooseNextMove();
         if (p != null) {
+          // Reset no pending move counter
+          noPendingMoveIteration=0;
           executePendingMove(p);
           continue;
         }

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

@@ -1821,8 +1821,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         checkOperation(OperationCategory.WRITE);
         writeLock(); // writelock is needed to set accesstime
       }
-      src = resolvePath(src, pathComponents);
       try {
+        src = resolvePath(src, pathComponents);
         if (isReadOp) {
           checkOperation(OperationCategory.READ);
         } else {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/BestEffortLongFile.java

@@ -112,6 +112,7 @@ public class BestEffortLongFile implements Closeable {
   public void close() throws IOException {
     if (ch != null) {
       ch.close();
+      ch = null;
     }
   }
 }

+ 18 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -1415,7 +1415,8 @@ public class DFSTestUtil {
   }
 
   /**
-   * Helper function to create a key in the Key Provider.
+   * Helper function to create a key in the Key Provider. Defaults
+   * to the first indexed NameNode's Key Provider.
    *
    * @param keyName The name of the key to create
    * @param cluster The cluster to create it in
@@ -1424,7 +1425,22 @@ public class DFSTestUtil {
   public static void createKey(String keyName, MiniDFSCluster cluster,
                                 Configuration conf)
           throws NoSuchAlgorithmException, IOException {
-    KeyProvider provider = cluster.getNameNode().getNamesystem().getProvider();
+    createKey(keyName, cluster, 0, conf);
+  }
+
+  /**
+   * Helper function to create a key in the Key Provider.
+   *
+   * @param keyName The name of the key to create
+   * @param cluster The cluster to create it in
+   * @param idx The NameNode index
+   * @param conf Configuration to use
+   */
+  public static void createKey(String keyName, MiniDFSCluster cluster,
+                               int idx, Configuration conf)
+      throws NoSuchAlgorithmException, IOException {
+    NameNode nn = cluster.getNameNode(idx);
+    KeyProvider provider = nn.getNamesystem().getProvider();
     final KeyProvider.Options options = KeyProvider.options(conf);
     options.setDescription(keyName);
     options.setBitLength(128);

+ 187 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -17,8 +17,14 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.RandomAccessFile;
+import java.io.StringReader;
+import java.io.StringWriter;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.List;
@@ -52,10 +58,15 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.EncryptionFaultInjector;
 import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
+import org.apache.hadoop.hdfs.tools.DFSck;
+import org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension.DelegationTokenExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 import org.apache.hadoop.io.Text;
@@ -78,6 +89,11 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import org.xml.sax.InputSource;
+import org.xml.sax.helpers.DefaultHandler;
+
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
 
 public class TestEncryptionZones {
 
@@ -374,6 +390,7 @@ public class TestEncryptionZones {
     final Path allPathFile = new Path(allPath, "file1");
     final Path nonEZDir = new Path(testRoot, "nonEZDir");
     final Path nonEZFile = new Path(nonEZDir, "file1");
+    final Path nonexistent = new Path("/nonexistent");
     final int len = 8192;
 
     fsWrapper.mkdir(testRoot, new FsPermission((short) 0777), true);
@@ -421,6 +438,9 @@ public class TestEncryptionZones {
           assertExceptionContains("Permission denied:", e);
         }
 
+        assertNull("expected null for nonexistent path",
+            userAdmin.getEncryptionZoneForPath(nonexistent));
+
         // Check operation with non-ez paths
         assertNull("expected null for non-ez path",
             userAdmin.getEncryptionZoneForPath(nonEZDir));
@@ -463,9 +483,15 @@ public class TestEncryptionZones {
     final Path testRoot = new Path("/tmp/TestEncryptionZones");
     final Path pathFoo = new Path(testRoot, "foo");
     final Path pathFooBaz = new Path(pathFoo, "baz");
+    final Path pathFooBazFile = new Path(pathFooBaz, "file");
+    final Path pathFooBar = new Path(pathFoo, "bar");
+    final Path pathFooBarFile = new Path(pathFooBar, "file");
+    final int len = 8192;
     wrapper.mkdir(pathFoo, FsPermission.getDirDefault(), true);
     dfsAdmin.createEncryptionZone(pathFoo, TEST_KEY);
     wrapper.mkdir(pathFooBaz, FsPermission.getDirDefault(), true);
+    DFSTestUtil.createFile(fs, pathFooBazFile, len, (short) 1, 0xFEED);
+    String contents = DFSTestUtil.readFile(fs, pathFooBazFile);
     try {
       wrapper.rename(pathFooBaz, testRoot);
     } catch (IOException e) {
@@ -473,6 +499,13 @@ public class TestEncryptionZones {
               " an encryption zone.", e
       );
     }
+
+    // Verify that we can rename dir and files within an encryption zone.
+    assertTrue(fs.rename(pathFooBaz, pathFooBar));
+    assertTrue("Rename of dir and file within ez failed",
+        !wrapper.exists(pathFooBaz) && wrapper.exists(pathFooBar));
+    assertEquals("Renamed file contents not the same",
+        contents, DFSTestUtil.readFile(fs, pathFooBarFile));
   }
 
   @Test(timeout = 60000)
@@ -806,4 +839,158 @@ public class TestEncryptionZones {
     Assert.assertEquals(tokens[1], testToken);
     Assert.assertEquals(1, creds.numberOfTokens());
   }
+
+  /**
+   * Test running fsck on a system with encryption zones.
+   */
+  @Test(timeout = 60000)
+  public void testFsckOnEncryptionZones() throws Exception {
+    final int len = 8196;
+    final Path zoneParent = new Path("/zones");
+    final Path zone1 = new Path(zoneParent, "zone1");
+    final Path zone1File = new Path(zone1, "file");
+    fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY);
+    DFSTestUtil.createFile(fs, zone1File, len, (short) 1, 0xFEED);
+    ByteArrayOutputStream bStream = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(bStream, true);
+    int errCode = ToolRunner.run(new DFSck(conf, out),
+        new String[]{ "/" });
+    assertEquals("Fsck ran with non-zero error code", 0, errCode);
+    String result = bStream.toString();
+    assertTrue("Fsck did not return HEALTHY status",
+        result.contains(NamenodeFsck.HEALTHY_STATUS));
+
+    // Run fsck directly on the encryption zone instead of root
+    errCode = ToolRunner.run(new DFSck(conf, out),
+        new String[]{ zoneParent.toString() });
+    assertEquals("Fsck ran with non-zero error code", 0, errCode);
+    result = bStream.toString();
+    assertTrue("Fsck did not return HEALTHY status",
+        result.contains(NamenodeFsck.HEALTHY_STATUS));
+  }
+
+  /**
+   * Test correctness of successive snapshot creation and deletion
+   * on a system with encryption zones.
+   */
+  @Test(timeout = 60000)
+  public void testSnapshotsOnEncryptionZones() throws Exception {
+    final int len = 8196;
+    final Path zoneParent = new Path("/zones");
+    final Path zone = new Path(zoneParent, "zone");
+    final Path zoneFile = new Path(zone, "zoneFile");
+    fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
+    dfsAdmin.allowSnapshot(zoneParent);
+    dfsAdmin.createEncryptionZone(zone, TEST_KEY);
+    DFSTestUtil.createFile(fs, zoneFile, len, (short) 1, 0xFEED);
+    String contents = DFSTestUtil.readFile(fs, zoneFile);
+    final Path snap1 = fs.createSnapshot(zoneParent);
+    final Path snap1Zone = new Path(snap1, zone.getName());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
+
+    // Now delete the encryption zone, recreate the dir, and take another snapshot
+    fsWrapper.delete(zone, true);
+    fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
+    final Path snap2 = fs.createSnapshot(zoneParent);
+    final Path snap2Zone = new Path(snap2, zone.getName());
+    assertNull("Expected null ez path",
+        dfsAdmin.getEncryptionZoneForPath(snap2Zone));
+
+    // Create the encryption zone again
+    dfsAdmin.createEncryptionZone(zone, TEST_KEY);
+    final Path snap3 = fs.createSnapshot(zoneParent);
+    final Path snap3Zone = new Path(snap3, zone.getName());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
+
+    // Verify contents of the snapshotted file
+    final Path snapshottedZoneFile = new Path(
+        snap1.toString() + "/" + zone.getName() + "/" + zoneFile.getName());
+    assertEquals("Contents of snapshotted file have changed unexpectedly",
+        contents, DFSTestUtil.readFile(fs, snapshottedZoneFile));
+
+    // Now delete the snapshots out of order and verify the zones are still correct
+    fs.deleteSnapshot(zoneParent, snap2.getName());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
+    fs.deleteSnapshot(zoneParent, snap1.getName());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
+  }
+
+  /**
+   * Verify symlinks can be created in encryption zones and that
+   * they function properly when the target is in the same
+   * or different ez.
+   */
+  @Test(timeout = 60000)
+  public void testEncryptionZonesWithSymlinks() throws Exception {
+    // Verify we can create an encryption zone over both link and target
+    final int len = 8192;
+    final Path parent = new Path("/parent");
+    final Path linkParent = new Path(parent, "symdir1");
+    final Path targetParent = new Path(parent, "symdir2");
+    final Path link = new Path(linkParent, "link");
+    final Path target = new Path(targetParent, "target");
+    fs.mkdirs(parent);
+    dfsAdmin.createEncryptionZone(parent, TEST_KEY);
+    fs.mkdirs(linkParent);
+    fs.mkdirs(targetParent);
+    DFSTestUtil.createFile(fs, target, len, (short)1, 0xFEED);
+    String content = DFSTestUtil.readFile(fs, target);
+    fs.createSymlink(target, link, false);
+    assertEquals("Contents read from link are not the same as target",
+        content, DFSTestUtil.readFile(fs, link));
+    fs.delete(parent, true);
+
+    // Now let's test when the symlink and target are in different
+    // encryption zones
+    fs.mkdirs(linkParent);
+    fs.mkdirs(targetParent);
+    dfsAdmin.createEncryptionZone(linkParent, TEST_KEY);
+    dfsAdmin.createEncryptionZone(targetParent, TEST_KEY);
+    DFSTestUtil.createFile(fs, target, len, (short)1, 0xFEED);
+    content = DFSTestUtil.readFile(fs, target);
+    fs.createSymlink(target, link, false);
+    assertEquals("Contents read from link are not the same as target",
+        content, DFSTestUtil.readFile(fs, link));
+    fs.delete(link, true);
+    fs.delete(target, true);
+  }
+
+  /**
+   * Test running the OfflineImageViewer on a system with encryption zones.
+   */
+  @Test(timeout = 60000)
+  public void testOfflineImageViewerOnEncryptionZones() throws Exception {
+    final int len = 8196;
+    final Path zoneParent = new Path("/zones");
+    final Path zone1 = new Path(zoneParent, "zone1");
+    final Path zone1File = new Path(zone1, "file");
+    fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY);
+    DFSTestUtil.createFile(fs, zone1File, len, (short) 1, 0xFEED);
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+    fs.saveNamespace();
+
+    File originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
+        .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0));
+    if (originalFsimage == null) {
+      throw new RuntimeException("Didn't generate or can't find fsimage");
+    }
+
+    // Run the XML OIV processor
+    StringWriter output = new StringWriter();
+    PrintWriter pw = new PrintWriter(output);
+    PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), pw);
+    v.visit(new RandomAccessFile(originalFsimage, "r"));
+    final String xml = output.getBuffer().toString();
+    SAXParser parser = SAXParserFactory.newInstance().newSAXParser();
+    parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
+  }
+
 }

+ 121 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java

@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Tests interaction of encryption zones with HA failover.
+ */
+public class TestEncryptionZonesWithHA {
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private NameNode nn0;
+  private NameNode nn1;
+  private DistributedFileSystem fs;
+  private HdfsAdmin dfsAdmin0;
+  private HdfsAdmin dfsAdmin1;
+  private FileSystemTestHelper fsHelper;
+  private File testRootDir;
+
+  private final String TEST_KEY = "testKey";
+
+
+  @Before
+  public void setupCluster() throws Exception {
+    conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    HAUtil.setAllowStandbyReads(conf, true);
+    fsHelper = new FileSystemTestHelper();
+    String testRoot = fsHelper.getTestRootDir();
+    testRootDir = new File(testRoot).getAbsoluteFile();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + testRootDir + "/test.jks"
+    );
+
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    cluster.waitActive();
+    cluster.transitionToActive(0);
+
+    fs = (DistributedFileSystem)HATestUtil.configureFailoverFs(cluster, conf);
+    DFSTestUtil.createKey(TEST_KEY, cluster, 0, conf);
+    DFSTestUtil.createKey(TEST_KEY, cluster, 1, conf);
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    dfsAdmin0 = new HdfsAdmin(cluster.getURI(0), conf);
+    dfsAdmin1 = new HdfsAdmin(cluster.getURI(1), conf);
+    KeyProviderCryptoExtension nn0Provider =
+        cluster.getNameNode(0).getNamesystem().getProvider();
+    fs.getClient().provider = nn0Provider;
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that encryption zones are properly tracked by the standby.
+   */
+  @Test(timeout = 60000)
+  public void testEncryptionZonesTrackedOnStandby() throws Exception {
+    final int len = 8196;
+    final Path dir = new Path("/enc");
+    final Path dirChild = new Path(dir, "child");
+    final Path dirFile = new Path(dir, "file");
+    fs.mkdir(dir, FsPermission.getDirDefault());
+    dfsAdmin0.createEncryptionZone(dir, TEST_KEY);
+    fs.mkdir(dirChild, FsPermission.getDirDefault());
+    DFSTestUtil.createFile(fs, dirFile, len, (short) 1, 0xFEED);
+    String contents = DFSTestUtil.readFile(fs, dirFile);
+
+    // Failover the current standby to active.
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    cluster.shutdownNameNode(0);
+    cluster.transitionToActive(1);
+
+    Assert.assertEquals("Got unexpected ez path", dir.toString(),
+        dfsAdmin1.getEncryptionZoneForPath(dir).getPath().toString());
+    Assert.assertEquals("Got unexpected ez path", dir.toString(),
+        dfsAdmin1.getEncryptionZoneForPath(dirChild).getPath().toString());
+    Assert.assertEquals("File contents after failover were changed",
+        contents, DFSTestUtil.readFile(fs, dirFile));
+  }
+
+}

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
@@ -80,4 +81,25 @@ public class TestRead {
     testEOF(cluster, 10000);   
     cluster.shutdown();
   }
+
+  /**
+   * Regression test for HDFS-7045.
+   * If deadlock happen, the test will time out.
+   * @throws Exception
+   */
+  @Test(timeout=60000)
+  public void testReadReservedPath() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(1).format(true).build();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      fs.open(new Path("/.reserved/.inodes/file"));
+      Assert.fail("Open a non existing file should fail.");
+    } catch (FileNotFoundException e) {
+      // Expected
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

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

@@ -279,6 +279,9 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-6075. HistoryServerFileSystemStateStore can create zero-length
     files (jlowe)
 
+    MAPREDUCE-6070. yarn.app.am.resource.mb/cpu-vcores affects uber mode but
+    is not documented (Tsuyoshi OZAWA via jlowe)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 6 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -508,7 +508,12 @@
   <description>Whether to enable the small-jobs "ubertask" optimization,
   which runs "sufficiently small" jobs sequentially within a single JVM.
   "Small" is defined by the following maxmaps, maxreduces, and maxbytes
-  settings.  Users may override this value.
+  settings. Note that configurations for application masters also affect
+  the "Small" definition - yarn.app.mapreduce.am.resource.mb must be
+  larger than both mapreduce.map.memory.mb and mapreduce.reduce.memory.mb,
+  and yarn.app.mapreduce.am.resource.cpu-vcores must be larger than
+  both mapreduce.map.cpu.vcores and mapreduce.reduce.cpu.vcores to enable
+  ubertask. Users may override this value.
   </description>
 </property>
 

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

@@ -323,6 +323,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2158. Fixed TestRMWebServicesAppsModification#testSingleAppKill test
     failure. (Varun Vasudev via jianhe)
 
+    YARN-2534. FairScheduler: Potential integer overflow calculating totalMaxShare. 
+    (Zhihai Xu via kasha)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java

@@ -124,11 +124,10 @@ public class ComputeFairShares {
     int totalMaxShare = 0;
     for (Schedulable sched : schedulables) {
       int maxShare = getResourceValue(sched.getMaxShare(), type);
-      if (maxShare == Integer.MAX_VALUE) {
-        totalMaxShare = Integer.MAX_VALUE;
+      totalMaxShare = (int) Math.min((long)maxShare + (long)totalMaxShare,
+          Integer.MAX_VALUE);
+      if (totalMaxShare == Integer.MAX_VALUE) {
         break;
-      } else {
-        totalMaxShare += maxShare;
       }
     }
 

+ 48 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -308,6 +308,54 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     }
   }
 
+  @Test
+  public void testFairShareWithMaxResources() throws IOException {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    // set queueA and queueB maxResources,
+    // the sum of queueA and queueB maxResources is more than
+    // Integer.MAX_VALUE.
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<maxResources>1073741824 mb 1000 vcores</maxResources>");
+    out.println("<weight>.25</weight>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<maxResources>1073741824 mb 1000 vcores</maxResources>");
+    out.println("<weight>.75</weight>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Add one big node (only care about aggregate capacity)
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    // Queue A wants 1 * 1024.
+    createSchedulingRequest(1 * 1024, "queueA", "user1");
+    // Queue B wants 6 * 1024
+    createSchedulingRequest(6 * 1024, "queueB", "user1");
+
+    scheduler.update();
+
+    FSLeafQueue queue = scheduler.getQueueManager().getLeafQueue(
+        "queueA", false);
+    // queueA's weight is 0.25, so its fair share should be 2 * 1024.
+    assertEquals(2 * 1024, queue.getFairShare().getMemory());
+    // queueB's weight is 0.75, so its fair share should be 6 * 1024.
+    queue = scheduler.getQueueManager().getLeafQueue(
+        "queueB", false);
+    assertEquals(6 * 1024, queue.getFairShare().getMemory());
+  }
+
   @Test
   public void testFairShareWithZeroWeight() throws IOException {
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);