Browse Source

Merge branch 'trunk' into HADOOP-12756

Kai Zheng 8 years ago
parent
commit
a57bba470b
100 changed files with 2273 additions and 5270 deletions
  1. 64 0
      hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/IncludePublicAnnotationsJDiffDoclet.java
  2. 1 1
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
  3. 99 122
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java
  4. 5 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  5. 1 1
      hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.7.2.xml
  6. 14 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java
  7. 222 62
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  8. 48 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.java
  9. 3 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
  10. 21 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
  11. 11 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  12. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
  13. 22 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
  14. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntryScope.java
  15. 22 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntryType.java
  16. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
  17. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
  18. 4 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
  19. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
  20. 9 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BloomMapFile.java
  21. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
  22. 7 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
  23. 13 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
  24. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
  25. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DefaultCodec.java
  26. 6 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java
  27. 10 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
  28. 91 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
  29. 62 26
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  30. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  31. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocksSocketFactory.java
  32. 2 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  33. 118 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosAuthException.java
  34. 46 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UGIExceptionMessages.java
  35. 47 58
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  36. 52 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java
  37. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
  38. 31 27
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
  39. 5 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/Hash.java
  40. 3 3
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  41. 2 1
      hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
  42. 58 19
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  43. 119 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
  44. 136 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  45. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java
  46. 75 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java
  47. 1 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDelegationTokenRenewer.java
  48. 4 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemInitialization.java
  49. 33 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java
  50. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
  51. 44 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
  52. 85 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  53. 23 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
  54. 4 29
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  55. 6 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
  56. 66 10
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  57. 2 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  58. 40 36
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  59. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
  60. 7 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  61. 109 37
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  62. 30 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  63. 0 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/CreateEncryptionZoneFlag.java
  64. 41 29
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  65. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsUtils.java
  66. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/package-info.java
  67. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
  68. 55 33
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
  69. 20 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java
  70. 0 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  71. 1 1
      hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.7.2.xml
  72. 0 66
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/README.txt
  73. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/dev-support/findbugsExcludeFile.xml
  74. 0 175
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
  75. 0 264
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
  76. 0 188
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java
  77. 0 893
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  78. 0 160
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/CurrentInprogress.java
  79. 0 217
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
  80. 0 103
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/MaxTxId.java
  81. 0 184
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/BKJMUtil.java
  82. 0 414
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
  83. 0 174
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperConfiguration.java
  84. 0 92
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
  85. 0 109
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
  86. 0 984
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java
  87. 0 167
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperSpeculativeRead.java
  88. 0 170
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java
  89. 0 160
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestCurrentInprogress.java
  90. 0 55
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
  91. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  92. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java
  93. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
  94. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  95. 55 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  96. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  97. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  98. 143 67
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
  99. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
  100. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

+ 64 - 0
hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/IncludePublicAnnotationsJDiffDoclet.java

@@ -0,0 +1,64 @@
+/*
+ * 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.classification.tools;
+
+import com.sun.javadoc.DocErrorReporter;
+import com.sun.javadoc.LanguageVersion;
+import com.sun.javadoc.RootDoc;
+
+import jdiff.JDiff;
+
+/**
+ * A <a href="http://java.sun.com/javase/6/docs/jdk/api/javadoc/doclet/">Doclet</a>
+ * that only includes class-level elements that are annotated with
+ * {@link org.apache.hadoop.classification.InterfaceAudience.Public}.
+ * Class-level elements with no annotation are excluded.
+ * In addition, all elements that are annotated with
+ * {@link org.apache.hadoop.classification.InterfaceAudience.Private} or
+ * {@link org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate}
+ * are also excluded.
+ * It delegates to the JDiff Doclet, and takes the same options.
+ */
+public class IncludePublicAnnotationsJDiffDoclet {
+
+  public static LanguageVersion languageVersion() {
+    return LanguageVersion.JAVA_1_5;
+  }
+
+  public static boolean start(RootDoc root) {
+    System.out.println(
+        IncludePublicAnnotationsJDiffDoclet.class.getSimpleName());
+    RootDocProcessor.treatUnannotatedClassesAsPrivate = true;
+    return JDiff.start(RootDocProcessor.process(root));
+  }
+
+  public static int optionLength(String option) {
+    Integer length = StabilityOptions.optionLength(option);
+    if (length != null) {
+      return length;
+    }
+    return JDiff.optionLength(option);
+  }
+
+  public static boolean validOptions(String[][] options,
+      DocErrorReporter reporter) {
+    StabilityOptions.validOptions(options, reporter);
+    String[][] filteredOptions = StabilityOptions.filterOptions(options);
+    return JDiff.validOptions(filteredOptions, reporter);
+  }
+}

+ 1 - 1
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java

@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
 public abstract class RolloverSignerSecretProvider
     extends SignerSecretProvider {
 
-  private static Logger LOG = LoggerFactory.getLogger(
+  static Logger LOG = LoggerFactory.getLogger(
     RolloverSignerSecretProvider.class);
   /**
    * Stores the currently valid secrets.  The current secret is the 0th element

+ 99 - 122
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java

@@ -17,7 +17,12 @@ import java.util.Arrays;
 import java.util.Properties;
 import java.util.Random;
 import javax.servlet.ServletContext;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.curator.test.TestingServer;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -25,7 +30,6 @@ import org.junit.Test;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -34,9 +38,14 @@ public class TestZKSignerSecretProvider {
   private TestingServer zkServer;
 
   // rollover every 2 sec
-  private final int timeout = 4000;
+  private final int timeout = 100;
   private final long rolloverFrequency = timeout / 2;
 
+  static final Log LOG = LogFactory.getLog(TestZKSignerSecretProvider.class);
+  {
+    LogManager.getLogger( RolloverSignerSecretProvider.LOG.getName() ).setLevel(Level.DEBUG);
+  }
+
   @Before
   public void setup() throws Exception {
     zkServer = new TestingServer();
@@ -60,8 +69,8 @@ public class TestZKSignerSecretProvider {
     byte[] secret2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secret1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secret3 = Long.toString(rand.nextLong()).getBytes();
-    ZKSignerSecretProvider secretProvider =
-        spy(new ZKSignerSecretProvider(seed));
+    MockZKSignerSecretProvider secretProvider =
+        spy(new MockZKSignerSecretProvider(seed));
     Properties config = new Properties();
     config.setProperty(
         ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
@@ -77,7 +86,8 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertArrayEquals(secret1, allSecrets[0]);
       Assert.assertNull(allSecrets[1]);
-      verify(secretProvider, timeout(timeout).times(1)).rollSecret();
+      verify(secretProvider, timeout(timeout).atLeastOnce()).rollSecret();
+      secretProvider.realRollSecret();
 
       currentSecret = secretProvider.getCurrentSecret();
       allSecrets = secretProvider.getAllSecrets();
@@ -85,7 +95,8 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertArrayEquals(secret2, allSecrets[0]);
       Assert.assertArrayEquals(secret1, allSecrets[1]);
-      verify(secretProvider, timeout(timeout).times(2)).rollSecret();
+      verify(secretProvider, timeout(timeout).atLeast(2)).rollSecret();
+      secretProvider.realRollSecret();
 
       currentSecret = secretProvider.getCurrentSecret();
       allSecrets = secretProvider.getAllSecrets();
@@ -93,128 +104,70 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertArrayEquals(secret3, allSecrets[0]);
       Assert.assertArrayEquals(secret2, allSecrets[1]);
-      verify(secretProvider, timeout(timeout).times(3)).rollSecret();
+      verify(secretProvider, timeout(timeout).atLeast(3)).rollSecret();
+      secretProvider.realRollSecret();
     } finally {
       secretProvider.destroy();
     }
   }
 
-  @Test
-  public void testMultipleInit() throws Exception {
-    // use the same seed so we can predict the RNG
-    long seedA = System.currentTimeMillis();
-    Random rand = new Random(seedA);
-    byte[] secretA2 = Long.toString(rand.nextLong()).getBytes();
-    byte[] secretA1 = Long.toString(rand.nextLong()).getBytes();
-    // use the same seed so we can predict the RNG
-    long seedB = System.currentTimeMillis() + rand.nextLong();
-    rand = new Random(seedB);
-    byte[] secretB2 = Long.toString(rand.nextLong()).getBytes();
-    byte[] secretB1 = Long.toString(rand.nextLong()).getBytes();
-    // use the same seed so we can predict the RNG
-    long seedC = System.currentTimeMillis() + rand.nextLong();
-    rand = new Random(seedC);
-    byte[] secretC2 = Long.toString(rand.nextLong()).getBytes();
-    byte[] secretC1 = Long.toString(rand.nextLong()).getBytes();
-    ZKSignerSecretProvider secretProviderA =
-        spy(new ZKSignerSecretProvider(seedA));
-    ZKSignerSecretProvider secretProviderB =
-        spy(new ZKSignerSecretProvider(seedB));
-    ZKSignerSecretProvider secretProviderC =
-        spy(new ZKSignerSecretProvider(seedC));
-    Properties config = new Properties();
-    config.setProperty(
-        ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
-        zkServer.getConnectString());
-    config.setProperty(ZKSignerSecretProvider.ZOOKEEPER_PATH,
-        "/secret");
-    try {
-      secretProviderA.init(config, getDummyServletContext(), rolloverFrequency);
-      secretProviderB.init(config, getDummyServletContext(), rolloverFrequency);
-      secretProviderC.init(config, getDummyServletContext(), rolloverFrequency);
-
-      byte[] currentSecretA = secretProviderA.getCurrentSecret();
-      byte[][] allSecretsA = secretProviderA.getAllSecrets();
-      byte[] currentSecretB = secretProviderB.getCurrentSecret();
-      byte[][] allSecretsB = secretProviderB.getAllSecrets();
-      byte[] currentSecretC = secretProviderC.getCurrentSecret();
-      byte[][] allSecretsC = secretProviderC.getAllSecrets();
-      Assert.assertArrayEquals(currentSecretA, currentSecretB);
-      Assert.assertArrayEquals(currentSecretB, currentSecretC);
-      Assert.assertEquals(2, allSecretsA.length);
-      Assert.assertEquals(2, allSecretsB.length);
-      Assert.assertEquals(2, allSecretsC.length);
-      Assert.assertArrayEquals(allSecretsA[0], allSecretsB[0]);
-      Assert.assertArrayEquals(allSecretsB[0], allSecretsC[0]);
-      Assert.assertNull(allSecretsA[1]);
-      Assert.assertNull(allSecretsB[1]);
-      Assert.assertNull(allSecretsC[1]);
-      char secretChosen = 'z';
-      if (Arrays.equals(secretA1, currentSecretA)) {
-        Assert.assertArrayEquals(secretA1, allSecretsA[0]);
-        secretChosen = 'A';
-      } else if (Arrays.equals(secretB1, currentSecretB)) {
-        Assert.assertArrayEquals(secretB1, allSecretsA[0]);
-        secretChosen = 'B';
-      }else if (Arrays.equals(secretC1, currentSecretC)) {
-        Assert.assertArrayEquals(secretC1, allSecretsA[0]);
-        secretChosen = 'C';
-      } else {
-        Assert.fail("It appears that they all agreed on the same secret, but "
-                + "not one of the secrets they were supposed to");
-      }
-      verify(secretProviderA, timeout(timeout).times(1)).rollSecret();
-      verify(secretProviderB, timeout(timeout).times(1)).rollSecret();
-      verify(secretProviderC, timeout(timeout).times(1)).rollSecret();
+  /**
+   * A hack to test ZKSignerSecretProvider.
+   * We want to test that ZKSignerSecretProvider.rollSecret() is periodically
+   * called at the expected frequency, but we want to exclude the
+   * race-condition.
+   */
+  private class MockZKSignerSecretProvider extends ZKSignerSecretProvider {
+    MockZKSignerSecretProvider(long seed) {
+      super(seed);
+    }
+    @Override
+    protected synchronized void rollSecret() {
+      // this is a no-op: simply used for Mockito to verify that rollSecret()
+      // is periodically called at the expected frequency
+    }
 
-      currentSecretA = secretProviderA.getCurrentSecret();
-      allSecretsA = secretProviderA.getAllSecrets();
-      currentSecretB = secretProviderB.getCurrentSecret();
-      allSecretsB = secretProviderB.getAllSecrets();
-      currentSecretC = secretProviderC.getCurrentSecret();
-      allSecretsC = secretProviderC.getAllSecrets();
-      Assert.assertArrayEquals(currentSecretA, currentSecretB);
-      Assert.assertArrayEquals(currentSecretB, currentSecretC);
-      Assert.assertEquals(2, allSecretsA.length);
-      Assert.assertEquals(2, allSecretsB.length);
-      Assert.assertEquals(2, allSecretsC.length);
-      Assert.assertArrayEquals(allSecretsA[0], allSecretsB[0]);
-      Assert.assertArrayEquals(allSecretsB[0], allSecretsC[0]);
-      Assert.assertArrayEquals(allSecretsA[1], allSecretsB[1]);
-      Assert.assertArrayEquals(allSecretsB[1], allSecretsC[1]);
-      // The second secret used is prechosen by whoever won the init; so it
-      // should match with whichever we saw before
-      if (secretChosen == 'A') {
-        Assert.assertArrayEquals(secretA2, currentSecretA);
-      } else if (secretChosen == 'B') {
-        Assert.assertArrayEquals(secretB2, currentSecretA);
-      } else if (secretChosen == 'C') {
-        Assert.assertArrayEquals(secretC2, currentSecretA);
-      }
-    } finally {
-      secretProviderC.destroy();
-      secretProviderB.destroy();
-      secretProviderA.destroy();
+    public void realRollSecret() {
+      // the test code manually calls ZKSignerSecretProvider.rollSecret()
+      // to update the state
+      super.rollSecret();
     }
   }
 
   @Test
-  public void testMultipleUnsychnronized() throws Exception {
+  public void testMultiple1() throws Exception {
+    testMultiple(1);
+  }
+
+  @Test
+  public void testMultiple2() throws Exception {
+    testMultiple(2);
+  }
+
+  /**
+   * @param order:
+   *            1: secretProviderA wins both realRollSecret races
+   *            2: secretProviderA wins 1st race, B wins 2nd
+   * @throws Exception
+   */
+  public void testMultiple(int order) throws Exception {
     long seedA = System.currentTimeMillis();
     Random rand = new Random(seedA);
     byte[] secretA2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretA1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretA3 = Long.toString(rand.nextLong()).getBytes();
+    byte[] secretA4 = Long.toString(rand.nextLong()).getBytes();
     // use the same seed so we can predict the RNG
     long seedB = System.currentTimeMillis() + rand.nextLong();
     rand = new Random(seedB);
     byte[] secretB2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretB1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretB3 = Long.toString(rand.nextLong()).getBytes();
-    ZKSignerSecretProvider secretProviderA =
-        spy(new ZKSignerSecretProvider(seedA));
-    ZKSignerSecretProvider secretProviderB =
-        spy(new ZKSignerSecretProvider(seedB));
+    byte[] secretB4 = Long.toString(rand.nextLong()).getBytes();
+    MockZKSignerSecretProvider secretProviderA =
+        spy(new MockZKSignerSecretProvider(seedA));
+    MockZKSignerSecretProvider secretProviderB =
+        spy(new MockZKSignerSecretProvider(seedB));
     Properties config = new Properties();
     config.setProperty(
         ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
@@ -223,14 +176,24 @@ public class TestZKSignerSecretProvider {
         "/secret");
     try {
       secretProviderA.init(config, getDummyServletContext(), rolloverFrequency);
+      secretProviderB.init(config, getDummyServletContext(), rolloverFrequency);
 
       byte[] currentSecretA = secretProviderA.getCurrentSecret();
       byte[][] allSecretsA = secretProviderA.getAllSecrets();
+      byte[] currentSecretB = secretProviderB.getCurrentSecret();
+      byte[][] allSecretsB = secretProviderB.getAllSecrets();
       Assert.assertArrayEquals(secretA1, currentSecretA);
+      Assert.assertArrayEquals(secretA1, currentSecretB);
       Assert.assertEquals(2, allSecretsA.length);
+      Assert.assertEquals(2, allSecretsB.length);
       Assert.assertArrayEquals(secretA1, allSecretsA[0]);
+      Assert.assertArrayEquals(secretA1, allSecretsB[0]);
       Assert.assertNull(allSecretsA[1]);
-      verify(secretProviderA, timeout(timeout).times(1)).rollSecret();
+      Assert.assertNull(allSecretsB[1]);
+      verify(secretProviderA, timeout(timeout).atLeastOnce()).rollSecret();
+      verify(secretProviderB, timeout(timeout).atLeastOnce()).rollSecret();
+      secretProviderA.realRollSecret();
+      secretProviderB.realRollSecret();
 
       currentSecretA = secretProviderA.getCurrentSecret();
       allSecretsA = secretProviderA.getAllSecrets();
@@ -238,18 +201,32 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertArrayEquals(secretA2, allSecretsA[0]);
       Assert.assertArrayEquals(secretA1, allSecretsA[1]);
-      Thread.sleep((rolloverFrequency / 5));
 
-      secretProviderB.init(config, getDummyServletContext(), rolloverFrequency);
-
-      byte[] currentSecretB = secretProviderB.getCurrentSecret();
-      byte[][] allSecretsB = secretProviderB.getAllSecrets();
+      currentSecretB = secretProviderB.getCurrentSecret();
+      allSecretsB = secretProviderB.getAllSecrets();
       Assert.assertArrayEquals(secretA2, currentSecretB);
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertArrayEquals(secretA2, allSecretsB[0]);
       Assert.assertArrayEquals(secretA1, allSecretsB[1]);
-      verify(secretProviderA, timeout(timeout).times(2)).rollSecret();
-      verify(secretProviderB, timeout(timeout).times(1)).rollSecret();
+      verify(secretProviderA, timeout(timeout).atLeast(2)).rollSecret();
+      verify(secretProviderB, timeout(timeout).atLeastOnce()).rollSecret();
+
+      switch (order) {
+        case 1:
+          secretProviderA.realRollSecret();
+          secretProviderB.realRollSecret();
+          secretProviderA.realRollSecret();
+          secretProviderB.realRollSecret();
+          break;
+        case 2:
+          secretProviderB.realRollSecret();
+          secretProviderA.realRollSecret();
+          secretProviderB.realRollSecret();
+          secretProviderA.realRollSecret();
+          break;
+        default:
+          throw new Exception("Invalid order selected");
+      }
 
       currentSecretA = secretProviderA.getCurrentSecret();
       allSecretsA = secretProviderA.getAllSecrets();
@@ -260,13 +237,13 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecretsB.length);
       Assert.assertArrayEquals(allSecretsA[0], allSecretsB[0]);
       Assert.assertArrayEquals(allSecretsA[1], allSecretsB[1]);
-      if (Arrays.equals(secretA3, currentSecretA)) {
-        Assert.assertArrayEquals(secretA3, allSecretsA[0]);
-      } else if (Arrays.equals(secretB3, currentSecretB)) {
-        Assert.assertArrayEquals(secretB3, allSecretsA[0]);
-      } else {
-        Assert.fail("It appears that they all agreed on the same secret, but "
-                + "not one of the secrets they were supposed to");
+      switch (order) {
+        case 1:
+          Assert.assertArrayEquals(secretA4, allSecretsA[0]);
+          break;
+        case 2:
+          Assert.assertArrayEquals(secretB4, allSecretsA[0]);
+          break;
       }
     } finally {
       secretProviderB.destroy();

+ 5 - 0
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -405,4 +405,9 @@
     <Bug pattern="NP_NULL_PARAM_DEREF"/>
   </Match>
 
+  <Match>
+    <Class name="org.apache.hadoop.ipc.ExternalCall"/>
+    <Filed name="done"/>
+    <Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER"/>
+  </Match>
 </FindBugsFilter>

File diff suppressed because it is too large
+ 1 - 1
hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.7.2.xml


+ 14 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java

@@ -70,11 +70,14 @@ public class ConfServlet extends HttpServlet {
       response.setContentType("application/json; charset=utf-8");
     }
 
+    String name = request.getParameter("name");
     Writer out = response.getWriter();
     try {
-      writeResponse(getConfFromContext(), out, format);
+      writeResponse(getConfFromContext(), out, format, name);
     } catch (BadFormatException bfe) {
       response.sendError(HttpServletResponse.SC_BAD_REQUEST, bfe.getMessage());
+    } catch (IllegalArgumentException iae) {
+      response.sendError(HttpServletResponse.SC_NOT_FOUND, iae.getMessage());
     }
     out.close();
   }
@@ -89,17 +92,23 @@ public class ConfServlet extends HttpServlet {
   /**
    * Guts of the servlet - extracted for easy testing.
    */
-  static void writeResponse(Configuration conf, Writer out, String format)
-    throws IOException, BadFormatException {
+  static void writeResponse(Configuration conf,
+      Writer out, String format, String propertyName)
+          throws IOException, IllegalArgumentException, BadFormatException {
     if (FORMAT_JSON.equals(format)) {
-      Configuration.dumpConfiguration(conf, out);
+      Configuration.dumpConfiguration(conf, propertyName, out);
     } else if (FORMAT_XML.equals(format)) {
-      conf.writeXml(out);
+      conf.writeXml(propertyName, out);
     } else {
       throw new BadFormatException("Bad format: " + format);
     }
   }
 
+  static void writeResponse(Configuration conf, Writer out, String format)
+      throws IOException, BadFormatException {
+    writeResponse(conf, out, format, null);
+  }
+
   public static class BadFormatException extends Exception {
     private static final long serialVersionUID = 1L;
 

+ 222 - 62
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -103,8 +103,9 @@ import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 
-/** 
+/**
  * Provides access to configuration parameters.
  *
  * <h4 id="Resources">Resources</h4>
@@ -2834,14 +2835,37 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     writeXml(new OutputStreamWriter(out, "UTF-8"));
   }
 
-  /** 
-   * Write out the non-default properties in this configuration to the given
-   * {@link Writer}.
-   * 
+  public void writeXml(Writer out) throws IOException {
+    writeXml(null, out);
+  }
+
+  /**
+   * Write out the non-default properties in this configuration to the
+   * given {@link Writer}.
+   *
+   * <li>
+   * When property name is not empty and the property exists in the
+   * configuration, this method writes the property and its attributes
+   * to the {@link Writer}.
+   * </li>
+   * <p>
+   *
+   * <li>
+   * When property name is null or empty, this method writes all the
+   * configuration properties and their attributes to the {@link Writer}.
+   * </li>
+   * <p>
+   *
+   * <li>
+   * When property name is not empty but the property doesn't exist in
+   * the configuration, this method throws an {@link IllegalArgumentException}.
+   * </li>
+   * <p>
    * @param out the writer to write to.
    */
-  public void writeXml(Writer out) throws IOException {
-    Document doc = asXmlDocument();
+  public void writeXml(String propertyName, Writer out)
+      throws IOException, IllegalArgumentException {
+    Document doc = asXmlDocument(propertyName);
 
     try {
       DOMSource source = new DOMSource(doc);
@@ -2861,62 +2885,180 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   /**
    * Return the XML DOM corresponding to this Configuration.
    */
-  private synchronized Document asXmlDocument() throws IOException {
+  private synchronized Document asXmlDocument(String propertyName)
+      throws IOException, IllegalArgumentException {
     Document doc;
     try {
-      doc =
-        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
+      doc = DocumentBuilderFactory
+          .newInstance()
+          .newDocumentBuilder()
+          .newDocument();
     } catch (ParserConfigurationException pe) {
       throw new IOException(pe);
     }
+
     Element conf = doc.createElement("configuration");
     doc.appendChild(conf);
     conf.appendChild(doc.createTextNode("\n"));
     handleDeprecation(); //ensure properties is set and deprecation is handled
-    for (Enumeration<Object> e = properties.keys(); e.hasMoreElements();) {
-      String name = (String)e.nextElement();
-      Object object = properties.get(name);
-      String value = null;
-      if (object instanceof String) {
-        value = (String) object;
-      }else {
-        continue;
-      }
-      Element propNode = doc.createElement("property");
-      conf.appendChild(propNode);
-
-      Element nameNode = doc.createElement("name");
-      nameNode.appendChild(doc.createTextNode(name));
-      propNode.appendChild(nameNode);
-
-      Element valueNode = doc.createElement("value");
-      valueNode.appendChild(doc.createTextNode(value));
-      propNode.appendChild(valueNode);
-
-      if (updatingResource != null) {
-        String[] sources = updatingResource.get(name);
-        if(sources != null) {
-          for(String s : sources) {
-            Element sourceNode = doc.createElement("source");
-            sourceNode.appendChild(doc.createTextNode(s));
-            propNode.appendChild(sourceNode);
+
+    if(!Strings.isNullOrEmpty(propertyName)) {
+      if (!properties.containsKey(propertyName)) {
+        // given property not found, illegal argument
+        throw new IllegalArgumentException("Property " +
+            propertyName + " not found");
+      } else {
+        // given property is found, write single property
+        appendXMLProperty(doc, conf, propertyName);
+        conf.appendChild(doc.createTextNode("\n"));
+      }
+    } else {
+      // append all elements
+      for (Enumeration<Object> e = properties.keys(); e.hasMoreElements();) {
+        appendXMLProperty(doc, conf, (String)e.nextElement());
+        conf.appendChild(doc.createTextNode("\n"));
+      }
+    }
+    return doc;
+  }
+
+  /**
+   *  Append a property with its attributes to a given {#link Document}
+   *  if the property is found in configuration.
+   *
+   * @param doc
+   * @param conf
+   * @param propertyName
+   */
+  private synchronized void appendXMLProperty(Document doc, Element conf,
+      String propertyName) {
+    // skip writing if given property name is empty or null
+    if (!Strings.isNullOrEmpty(propertyName)) {
+      String value = properties.getProperty(propertyName);
+      if (value != null) {
+        Element propNode = doc.createElement("property");
+        conf.appendChild(propNode);
+
+        Element nameNode = doc.createElement("name");
+        nameNode.appendChild(doc.createTextNode(propertyName));
+        propNode.appendChild(nameNode);
+
+        Element valueNode = doc.createElement("value");
+        valueNode.appendChild(doc.createTextNode(
+            properties.getProperty(propertyName)));
+        propNode.appendChild(valueNode);
+
+        Element finalNode = doc.createElement("final");
+        finalNode.appendChild(doc.createTextNode(
+            String.valueOf(finalParameters.contains(propertyName))));
+        propNode.appendChild(finalNode);
+
+        if (updatingResource != null) {
+          String[] sources = updatingResource.get(propertyName);
+          if(sources != null) {
+            for(String s : sources) {
+              Element sourceNode = doc.createElement("source");
+              sourceNode.appendChild(doc.createTextNode(s));
+              propNode.appendChild(sourceNode);
+            }
           }
         }
       }
-      
-      conf.appendChild(doc.createTextNode("\n"));
     }
-    return doc;
   }
 
   /**
-   *  Writes out all the parameters and their properties (final and resource) to
-   *  the given {@link Writer}
-   *  The format of the output would be 
-   *  { "properties" : [ {key1,value1,key1.isFinal,key1.resource}, {key2,value2,
-   *  key2.isFinal,key2.resource}... ] } 
-   *  It does not output the parameters of the configuration object which is 
-   *  loaded from an input stream.
+   *  Writes properties and their attributes (final and resource)
+   *  to the given {@link Writer}.
+   *
+   *  <li>
+   *  When propertyName is not empty, and the property exists
+   *  in the configuration, the format of the output would be,
+   *  <pre>
+   *  {
+   *    "property": {
+   *      "key" : "key1",
+   *      "value" : "value1",
+   *      "isFinal" : "key1.isFinal",
+   *      "resource" : "key1.resource"
+   *    }
+   *  }
+   *  </pre>
+   *  </li>
+   *
+   *  <li>
+   *  When propertyName is null or empty, it behaves same as
+   *  {@link #dumpConfiguration(Configuration, Writer)}, the
+   *  output would be,
+   *  <pre>
+   *  { "properties" :
+   *      [ { key : "key1",
+   *          value : "value1",
+   *          isFinal : "key1.isFinal",
+   *          resource : "key1.resource" },
+   *        { key : "key2",
+   *          value : "value2",
+   *          isFinal : "ke2.isFinal",
+   *          resource : "key2.resource" }
+   *       ]
+   *   }
+   *  </pre>
+   *  </li>
+   *
+   *  <li>
+   *  When propertyName is not empty, and the property is not
+   *  found in the configuration, this method will throw an
+   *  {@link IllegalArgumentException}.
+   *  </li>
+   *  <p>
+   * @param config the configuration
+   * @param propertyName property name
+   * @param out the Writer to write to
+   * @throws IOException
+   * @throws IllegalArgumentException when property name is not
+   *   empty and the property is not found in configuration
+   **/
+  public static void dumpConfiguration(Configuration config,
+      String propertyName, Writer out) throws IOException {
+    if(Strings.isNullOrEmpty(propertyName)) {
+      dumpConfiguration(config, out);
+    } else if (Strings.isNullOrEmpty(config.get(propertyName))) {
+      throw new IllegalArgumentException("Property " +
+          propertyName + " not found");
+    } else {
+      JsonFactory dumpFactory = new JsonFactory();
+      JsonGenerator dumpGenerator = dumpFactory.createJsonGenerator(out);
+      dumpGenerator.writeStartObject();
+      dumpGenerator.writeFieldName("property");
+      appendJSONProperty(dumpGenerator, config, propertyName);
+      dumpGenerator.writeEndObject();
+      dumpGenerator.flush();
+    }
+  }
+
+  /**
+   *  Writes out all properties and their attributes (final and resource) to
+   *  the given {@link Writer}, the format of the output would be,
+   *
+   *  <pre>
+   *  { "properties" :
+   *      [ { key : "key1",
+   *          value : "value1",
+   *          isFinal : "key1.isFinal",
+   *          resource : "key1.resource" },
+   *        { key : "key2",
+   *          value : "value2",
+   *          isFinal : "ke2.isFinal",
+   *          resource : "key2.resource" }
+   *       ]
+   *   }
+   *  </pre>
+   *
+   *  It does not output the properties of the configuration object which
+   *  is loaded from an input stream.
+   *  <p>
+   *
+   * @param config the configuration
    * @param out the Writer to write to
    * @throws IOException
    */
@@ -2930,29 +3072,47 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     dumpGenerator.flush();
     synchronized (config) {
       for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
-        dumpGenerator.writeStartObject();
-        dumpGenerator.writeStringField("key", (String) item.getKey());
-        dumpGenerator.writeStringField("value", 
-                                       config.get((String) item.getKey()));
-        dumpGenerator.writeBooleanField("isFinal",
-                                        config.finalParameters.contains(item.getKey()));
-        String[] resources = config.updatingResource.get(item.getKey());
-        String resource = UNKNOWN_RESOURCE;
-        if(resources != null && resources.length > 0) {
-          resource = resources[0];
-        }
-        dumpGenerator.writeStringField("resource", resource);
-        dumpGenerator.writeEndObject();
+        appendJSONProperty(dumpGenerator,
+            config,
+            item.getKey().toString());
       }
     }
     dumpGenerator.writeEndArray();
     dumpGenerator.writeEndObject();
     dumpGenerator.flush();
   }
-  
+
+  /**
+   * Write property and its attributes as json format to given
+   * {@link JsonGenerator}.
+   *
+   * @param jsonGen json writer
+   * @param config configuration
+   * @param name property name
+   * @throws IOException
+   */
+  private static void appendJSONProperty(JsonGenerator jsonGen,
+      Configuration config, String name) throws IOException {
+    // skip writing if given property name is empty or null
+    if(!Strings.isNullOrEmpty(name) && jsonGen != null) {
+      jsonGen.writeStartObject();
+      jsonGen.writeStringField("key", name);
+      jsonGen.writeStringField("value", config.get(name));
+      jsonGen.writeBooleanField("isFinal",
+          config.finalParameters.contains(name));
+      String[] resources = config.updatingResource.get(name);
+      String resource = UNKNOWN_RESOURCE;
+      if(resources != null && resources.length > 0) {
+        resource = resources[0];
+      }
+      jsonGen.writeStringField("resource", resource);
+      jsonGen.writeEndObject();
+    }
+  }
+
   /**
    * Get the {@link ClassLoader} for this job.
-   * 
+   *
    * @return the correct class loader.
    */
   public ClassLoader getClassLoader() {

+ 48 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.java

@@ -0,0 +1,48 @@
+/**
+ * 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.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Fast but inaccurate class to tell how much space HDFS is using.
+ * This class makes the assumption that the entire mount is used for
+ * HDFS and that no two hdfs data dirs are on the same disk.
+ *
+ * To use set fs.getspaceused.classname
+ * to org.apache.hadoop.fs.DFCachingGetSpaceUsed in your core-site.xml
+ *
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class DFCachingGetSpaceUsed extends CachingGetSpaceUsed {
+  private final DF df;
+
+  public DFCachingGetSpaceUsed(Builder builder) throws IOException {
+    super(builder);
+    this.df = new DF(builder.getPath(), builder.getInterval());
+  }
+
+  @Override
+  protected void refresh() {
+    this.used.set(df.getUsed());
+  }
+}

+ 3 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java

@@ -31,12 +31,13 @@ import java.io.IOException;
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public class DU extends CachingGetSpaceUsed {
-  private DUShell duShell;
+  private final DUShell duShell;
 
   @VisibleForTesting
-  public DU(File path, long interval, long jitter, long initialUsed)
+   public DU(File path, long interval, long jitter, long initialUsed)
       throws IOException {
     super(path, interval, jitter, initialUsed);
+    this.duShell = new DUShell();
   }
 
   public DU(CachingGetSpaceUsed.Builder builder) throws IOException {
@@ -48,9 +49,6 @@ public class DU extends CachingGetSpaceUsed {
 
   @Override
   protected synchronized void refresh() {
-    if (duShell == null) {
-      duShell = new DUShell();
-    }
     try {
       duShell.startRefresh();
     } catch (IOException ioe) {

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java

@@ -121,4 +121,25 @@ public class FileEncryptionInfo {
     builder.append("}");
     return builder.toString();
   }
+
+  /**
+   * A frozen version of {@link #toString()} to be backward compatible.
+   * When backward compatibility is not needed, use {@link #toString()}, which
+   * provides more info and is supposed to evolve.
+   * Don't change this method except for major revisions.
+   *
+   * NOTE:
+   * Currently this method is used by CLI for backward compatibility.
+   */
+  public String toStringStable() {
+    StringBuilder builder = new StringBuilder("{");
+    builder.append("cipherSuite: " + cipherSuite);
+    builder.append(", cryptoProtocolVersion: " + version);
+    builder.append(", edek: " + Hex.encodeHexString(edek));
+    builder.append(", iv: " + Hex.encodeHexString(iv));
+    builder.append(", keyName: " + keyName);
+    builder.append(", ezKeyVersionName: " + ezKeyVersionName);
+    builder.append("}");
+    return builder.toString();
+  }
 }

+ 11 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -2858,7 +2858,15 @@ public abstract class FileSystem extends Configured implements Closeable {
                   ClassUtil.findContainingJar(fs.getClass()), e);
             }
           } catch (ServiceConfigurationError ee) {
-            LOG.warn("Cannot load filesystem", ee);
+            LOG.warn("Cannot load filesystem: " + ee);
+            Throwable cause = ee.getCause();
+            // print all the nested exception messages
+            while (cause != null) {
+              LOG.warn(cause.toString());
+              cause = cause.getCause();
+            }
+            // and at debug: the full stack
+            LOG.debug("Stack Trace", ee);
           }
         }
         FILE_SYSTEMS_LOADED = true;
@@ -2944,7 +2952,8 @@ public abstract class FileSystem extends Configured implements Closeable {
         }
         fs.key = key;
         map.put(key, fs);
-        if (conf.getBoolean("fs.automatic.close", true)) {
+        if (conf.getBoolean(
+            FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
           toAutoClose.add(key);
         }
         return fs;

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java

@@ -105,13 +105,13 @@ public class FTPFileSystem extends FileSystem {
     // get port information from uri, (overrides info in conf)
     int port = uri.getPort();
     port = (port == -1) ? FTP.DEFAULT_PORT : port;
-    conf.setInt("fs.ftp.host.port", port);
+    conf.setInt(FS_FTP_HOST_PORT, port);
 
     // get user/password information from URI (overrides info in conf)
     String userAndPassword = uri.getUserInfo();
     if (userAndPassword == null) {
-      userAndPassword = (conf.get("fs.ftp.user." + host, null) + ":" + conf
-          .get("fs.ftp.password." + host, null));
+      userAndPassword = (conf.get(FS_FTP_USER_PREFIX + host, null) + ":" + conf
+          .get(FS_FTP_PASSWORD_PREFIX + host, null));
     }
     String[] userPasswdInfo = userAndPassword.split(":");
     Preconditions.checkState(userPasswdInfo.length > 1,

+ 22 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.util.StringUtils;
  * to create a new instance.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public class AclEntry {
   private final AclEntryType type;
   private final String name;
@@ -100,13 +100,29 @@ public class AclEntry {
   }
 
   @Override
+  @InterfaceStability.Unstable
   public String toString() {
+    // This currently just delegates to the stable string representation, but it
+    // is permissible for the output of this method to change across versions.
+    return toStringStable();
+  }
+
+  /**
+   * Returns a string representation guaranteed to be stable across versions to
+   * satisfy backward compatibility requirements, such as for shell command
+   * output or serialization.  The format of this string representation matches
+   * what is expected by the {@link #parseAclSpec(String, boolean)} and
+   * {@link #parseAclEntry(String, boolean)} methods.
+   *
+   * @return stable, backward compatible string representation
+   */
+  public String toStringStable() {
     StringBuilder sb = new StringBuilder();
     if (scope == AclEntryScope.DEFAULT) {
       sb.append("default:");
     }
     if (type != null) {
-      sb.append(StringUtils.toLowerCase(type.toString()));
+      sb.append(StringUtils.toLowerCase(type.toStringStable()));
     }
     sb.append(':');
     if (name != null) {
@@ -203,6 +219,8 @@ public class AclEntry {
   /**
    * Parses a string representation of an ACL spec into a list of AclEntry
    * objects. Example: "user::rwx,user:foo:rw-,group::r--,other::---"
+   * The expected format of ACL entries in the string parameter is the same
+   * format produced by the {@link #toStringStable()} method.
    * 
    * @param aclSpec
    *          String representation of an ACL spec.
@@ -228,6 +246,8 @@ public class AclEntry {
 
   /**
    * Parses a string representation of an ACL into a AclEntry object.<br>
+   * The expected format of ACL entries in the string parameter is the same
+   * format produced by the {@link #toStringStable()} method.
    * 
    * @param aclStr
    *          String representation of an ACL.<br>

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntryScope.java

@@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  * Specifies the scope or intended usage of an ACL entry.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public enum AclEntryScope {
   /**
    * An ACL entry that is inspected during permission checks to enforce

+ 22 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntryType.java

@@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  * Specifies the type of an ACL entry.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public enum AclEntryType {
   /**
    * An ACL entry applied to a specific user.  These ACL entries can be unnamed,
@@ -55,4 +55,25 @@ public enum AclEntryType {
    * of the more specific ACL entry types.
    */
   OTHER;
+
+  @Override
+  @InterfaceStability.Unstable
+  public String toString() {
+    // This currently just delegates to the stable string representation, but it
+    // is permissible for the output of this method to change across versions.
+    return toStringStable();
+  }
+
+  /**
+   * Returns a string representation guaranteed to be stable across versions to
+   * satisfy backward compatibility requirements, such as for shell command
+   * output or serialization.
+   *
+   * @return stable, backward compatible string representation
+   */
+  public String toStringStable() {
+    // The base implementation uses the enum value names, which are public API
+    // and therefore stable.
+    return super.toString();
+  }
 }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java

@@ -31,7 +31,7 @@ import com.google.common.collect.Lists;
  * instances are immutable. Use a {@link Builder} to create a new instance.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public class AclStatus {
   private final String owner;
   private final String group;

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java

@@ -117,7 +117,7 @@ class AclCommands extends FsCommand {
       }
       if (AclUtil.isMinimalAcl(entries)) {
         for (AclEntry entry: entries) {
-          out.println(entry);
+          out.println(entry.toStringStable());
         }
       } else {
         for (AclEntry entry: entries) {
@@ -145,10 +145,10 @@ class AclCommands extends FsCommand {
           out.println(String.format("%s\t#effective:%s", entry,
             effectivePerm.SYMBOL));
         } else {
-          out.println(entry);
+          out.println(entry.toStringStable());
         }
       } else {
-        out.println(entry);
+        out.println(entry.toStringStable());
       }
     }
   }

+ 4 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -46,6 +46,8 @@ import org.apache.hadoop.fs.permission.AclUtil;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
 
@@ -497,7 +499,8 @@ abstract class CommandWithDestination extends FsCommand {
                         FsPermission.getFileDefault().applyUMask(
                             FsPermission.getUMask(getConf())),
                         createFlags,
-                        getConf().getInt("io.file.buffer.size", 4096),
+                        getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
+                            IO_FILE_BUFFER_SIZE_DEFAULT),
                         lazyPersist ? 1 : getDefaultReplication(item.path),
                         getDefaultBlockSize(),
                         null,

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java

@@ -83,7 +83,7 @@ import org.apache.hadoop.util.Time;
  * ViewFs is specified with the following URI: <b>viewfs:///</b> 
  * <p>
  * To use viewfs one would typically set the default file system in the
- * config  (i.e. fs.default.name< = viewfs:///) along with the
+ * config  (i.e. fs.defaultFS < = viewfs:///) along with the
  * mount table config variables as described below. 
  * 
  * <p>

+ 9 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BloomMapFile.java

@@ -37,6 +37,11 @@ import org.apache.hadoop.util.bloom.Filter;
 import org.apache.hadoop.util.bloom.Key;
 import org.apache.hadoop.util.hash.Hash;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_MAPFILE_BLOOM_ERROR_RATE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_MAPFILE_BLOOM_ERROR_RATE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_MAPFILE_BLOOM_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_MAPFILE_BLOOM_SIZE_KEY;
+
 /**
  * This class extends {@link MapFile} and provides very much the same
  * functionality. However, it uses dynamic Bloom filters to provide
@@ -159,13 +164,15 @@ public class BloomMapFile {
     }
 
     private synchronized void initBloomFilter(Configuration conf) {
-      numKeys = conf.getInt("io.mapfile.bloom.size", 1024 * 1024);
+      numKeys = conf.getInt(
+          IO_MAPFILE_BLOOM_SIZE_KEY, IO_MAPFILE_BLOOM_SIZE_DEFAULT);
       // vector size should be <code>-kn / (ln(1 - c^(1/k)))</code> bits for
       // single key, where <code> is the number of hash functions,
       // <code>n</code> is the number of keys and <code>c</code> is the desired
       // max. error rate.
       // Our desired error rate is by default 0.005, i.e. 0.5%
-      float errorRate = conf.getFloat("io.mapfile.bloom.error.rate", 0.005f);
+      float errorRate = conf.getFloat(
+          IO_MAPFILE_BLOOM_ERROR_RATE_KEY, IO_MAPFILE_BLOOM_ERROR_RATE_DEFAULT);
       vectorSize = (int)Math.ceil((double)(-HASH_COUNT * numKeys) /
           Math.log(1.0 - Math.pow(errorRate, 1.0/HASH_COUNT)));
       bloomFilter = new DynamicBloomFilter(vectorSize, HASH_COUNT,

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java

@@ -38,6 +38,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ChunkedArrayList;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
  * An utility class for I/O related functionality. 
  */
@@ -105,7 +108,8 @@ public class IOUtils {
    */
   public static void copyBytes(InputStream in, OutputStream out, Configuration conf)
     throws IOException {
-    copyBytes(in, out, conf.getInt("io.file.buffer.size", 4096), true);
+    copyBytes(in, out, conf.getInt(
+        IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT), true);
   }
   
   /**
@@ -119,7 +123,8 @@ public class IOUtils {
    */
   public static void copyBytes(InputStream in, OutputStream out, Configuration conf, boolean close)
     throws IOException {
-    copyBytes(in, out, conf.getInt("io.file.buffer.size", 4096),  close);
+    copyBytes(in, out, conf.getInt(
+        IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT),  close);
   }
 
   /**

+ 7 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java

@@ -38,6 +38,9 @@ import org.apache.hadoop.util.Options;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_MAP_INDEX_SKIP_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_MAP_INDEX_SKIP_KEY;
+
 /** A file-based map from keys to values.
  * 
  * <p>A map is a directory containing two files, the <code>data</code> file,
@@ -395,7 +398,8 @@ public class MapFile {
         Options.getOption(ComparatorOption.class, opts);
       WritableComparator comparator =
         comparatorOption == null ? null : comparatorOption.getValue();
-      INDEX_SKIP = conf.getInt("io.map.index.skip", 0);
+      INDEX_SKIP = conf.getInt(
+          IO_MAP_INDEX_SKIP_KEY, IO_MAP_INDEX_SKIP_DEFAULT);
       open(dir, comparator, conf, opts);
     }
  
@@ -990,8 +994,8 @@ public class MapFile {
             reader.getKeyClass().asSubclass(WritableComparable.class),
             reader.getValueClass());
 
-      WritableComparable key = ReflectionUtils.newInstance(reader.getKeyClass()
-        .asSubclass(WritableComparable.class), conf);
+      WritableComparable<?> key = ReflectionUtils.newInstance(
+          reader.getKeyClass().asSubclass(WritableComparable.class), conf);
       Writable value = ReflectionUtils.newInstance(reader.getValueClass()
         .asSubclass(Writable.class), conf);
 

+ 13 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java

@@ -51,6 +51,13 @@ import org.apache.hadoop.util.MergeSort;
 import org.apache.hadoop.util.PriorityQueue;
 import org.apache.hadoop.util.Time;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SEQFILE_COMPRESS_BLOCKSIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SEQFILE_COMPRESS_BLOCKSIZE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SKIP_CHECKSUM_ERRORS_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SKIP_CHECKSUM_ERRORS_KEY;
+
 /** 
  * <code>SequenceFile</code>s are flat files consisting of binary key/value 
  * pairs.
@@ -1513,7 +1520,9 @@ public class SequenceFile {
                         Option... options) throws IOException {
       super(conf, options);
       compressionBlockSize = 
-        conf.getInt("io.seqfile.compress.blocksize", 1000000);
+        conf.getInt(IO_SEQFILE_COMPRESS_BLOCKSIZE_KEY,
+            IO_SEQFILE_COMPRESS_BLOCKSIZE_DEFAULT
+        );
       keySerializer.close();
       keySerializer.open(keyBuffer);
       uncompressedValSerializer.close();
@@ -1637,7 +1646,7 @@ public class SequenceFile {
 
   /** Get the configured buffer size */
   private static int getBufferSize(Configuration conf) {
-    return conf.getInt("io.file.buffer.size", 4096);
+    return conf.getInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT);
   }
 
   /** Reads key/value pairs from a sequence-format file. */
@@ -2655,7 +2664,8 @@ public class SequenceFile {
 
     private void handleChecksumException(ChecksumException e)
       throws IOException {
-      if (this.conf.getBoolean("io.skip.checksum.errors", false)) {
+      if (this.conf.getBoolean(
+          IO_SKIP_CHECKSUM_ERRORS_KEY, IO_SKIP_CHECKSUM_ERRORS_DEFAULT)) {
         LOG.warn("Bad checksum at "+getPosition()+". Skipping entries.");
         sync(getPosition()+this.conf.getInt("io.bytes.per.checksum", 512));
       } else {

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java

@@ -35,6 +35,9 @@ import org.apache.hadoop.io.compress.bzip2.CBZip2InputStream;
 import org.apache.hadoop.io.compress.bzip2.CBZip2OutputStream;
 import org.apache.hadoop.io.compress.bzip2.Bzip2Factory;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
  * This class provides output and input streams for bzip2 compression
  * and decompression.  It uses the native bzip2 library on the system
@@ -120,7 +123,8 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
       Compressor compressor) throws IOException {
     return Bzip2Factory.isNativeBzip2Loaded(conf) ?
       new CompressorStream(out, compressor, 
-                           conf.getInt("io.file.buffer.size", 4*1024)) :
+                           conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                   IO_FILE_BUFFER_SIZE_DEFAULT)) :
       new BZip2CompressionOutputStream(out);
   }
 
@@ -174,7 +178,8 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
       Decompressor decompressor) throws IOException {
     return Bzip2Factory.isNativeBzip2Loaded(conf) ? 
       new DecompressorStream(in, decompressor,
-                             conf.getInt("io.file.buffer.size", 4*1024)) :
+                             conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                 IO_FILE_BUFFER_SIZE_DEFAULT)) :
       new BZip2CompressionInputStream(in);
   }
 

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DefaultCodec.java

@@ -31,6 +31,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.zlib.ZlibDecompressor;
 import org.apache.hadoop.io.compress.zlib.ZlibFactory;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class DefaultCodec implements Configurable, CompressionCodec, DirectDecompressionCodec {
@@ -60,7 +63,8 @@ public class DefaultCodec implements Configurable, CompressionCodec, DirectDecom
                                                     Compressor compressor) 
   throws IOException {
     return new CompressorStream(out, compressor, 
-                                conf.getInt("io.file.buffer.size", 4*1024));
+                                conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                        IO_FILE_BUFFER_SIZE_DEFAULT));
   }
 
   @Override
@@ -85,7 +89,8 @@ public class DefaultCodec implements Configurable, CompressionCodec, DirectDecom
                                                   Decompressor decompressor) 
   throws IOException {
     return new DecompressorStream(in, decompressor, 
-                                  conf.getInt("io.file.buffer.size", 4*1024));
+                                  conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                      IO_FILE_BUFFER_SIZE_DEFAULT));
   }
 
   @Override

+ 6 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.io.compress.zlib.*;
 import org.apache.hadoop.io.compress.zlib.ZlibDecompressor.ZlibDirectDecompressor;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 /**
@@ -172,8 +174,8 @@ public class GzipCodec extends DefaultCodec {
   throws IOException {
     return (compressor != null) ?
                new CompressorStream(out, compressor,
-                                    conf.getInt("io.file.buffer.size", 
-                                                4*1024)) :
+                                    conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                            IO_FILE_BUFFER_SIZE_DEFAULT)) :
                createOutputStream(out);
   }
 
@@ -206,7 +208,8 @@ public class GzipCodec extends DefaultCodec {
       decompressor = createDecompressor();  // always succeeds (or throws)
     }
     return new DecompressorStream(in, decompressor,
-                                  conf.getInt("io.file.buffer.size", 4*1024));
+                                  conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                      IO_FILE_BUFFER_SIZE_DEFAULT));
   }
 
   @Override

+ 10 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java

@@ -36,6 +36,10 @@ import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
  * Compression related stuff.
  */
@@ -124,7 +128,8 @@ final class Compression {
         } else {
           bis1 = downStream;
         }
-        conf.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
+        conf.setInt(IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY,
+            IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT);
         CompressionInputStream cis =
             codec.createInputStream(bis1, decompressor);
         BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
@@ -146,7 +151,8 @@ final class Compression {
         } else {
           bos1 = downStream;
         }
-        conf.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
+        conf.setInt(IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY,
+            IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT);
         CompressionOutputStream cos =
             codec.createOutputStream(bos1, compressor);
         BufferedOutputStream bos2 =
@@ -175,7 +181,7 @@ final class Compression {
           int downStreamBufferSize) throws IOException {
         // Set the internal buffer size to read from down stream.
         if (downStreamBufferSize > 0) {
-          codec.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
+          codec.getConf().setInt(IO_FILE_BUFFER_SIZE_KEY, downStreamBufferSize);
         }
         CompressionInputStream cis =
             codec.createInputStream(downStream, decompressor);
@@ -193,7 +199,7 @@ final class Compression {
         } else {
           bos1 = downStream;
         }
-        codec.getConf().setInt("io.file.buffer.size", 32 * 1024);
+        codec.getConf().setInt(IO_FILE_BUFFER_SIZE_KEY, 32 * 1024);
         CompressionOutputStream cos =
             codec.createOutputStream(bos1, compressor);
         BufferedOutputStream bos2 =

+ 91 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java

@@ -0,0 +1,91 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.ipc.Server.Call;
+import org.apache.hadoop.security.UserGroupInformation;
+
+public abstract class ExternalCall<T> extends Call {
+  private final PrivilegedExceptionAction<T> action;
+  private final AtomicBoolean done = new AtomicBoolean();
+  private T result;
+  private Throwable error;
+
+  public ExternalCall(PrivilegedExceptionAction<T> action) {
+    this.action = action;
+  }
+
+  public abstract UserGroupInformation getRemoteUser();
+
+  public final T get() throws IOException, InterruptedException {
+    waitForCompletion();
+    if (error != null) {
+      if (error instanceof IOException) {
+        throw (IOException)error;
+      } else {
+        throw new IOException(error);
+      }
+    }
+    return result;
+  }
+
+  // wait for response to be triggered to support postponed calls
+  private void waitForCompletion() throws InterruptedException {
+    synchronized(done) {
+      while (!done.get()) {
+        try {
+          done.wait();
+        } catch (InterruptedException ie) {
+          if (Thread.interrupted()) {
+            throw ie;
+          }
+        }
+      }
+    }
+  }
+
+  boolean isDone() {
+    return done.get();
+  }
+
+  // invoked by ipc handler
+  @Override
+  public final Void run() throws IOException {
+    try {
+      result = action.run();
+      sendResponse();
+    } catch (Throwable t) {
+      abortResponse(t);
+    }
+    return null;
+  }
+
+  @Override
+  final void doResponse(Throwable t) {
+    synchronized(done) {
+      error = t;
+      done.set(true);
+      done.notify();
+    }
+  }
+}

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

@@ -110,6 +110,7 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -383,6 +384,11 @@ public abstract class Server {
     return (call != null) ? call.getRemoteUser() : null;
   }
 
+  public static String getProtocol() {
+    Call call = CurCall.get();
+    return (call != null) ? call.getProtocol() : null;
+  }
+
   /** Return true if the invocation was through an RPC.
    */
   public static boolean isRpcInvocation() {
@@ -671,6 +677,11 @@ public abstract class Server {
     private int priorityLevel;
     // the priority level assigned by scheduler, 0 by default
 
+    Call() {
+      this(RpcConstants.INVALID_CALL_ID, RpcConstants.INVALID_RETRY_COUNT,
+        RPC.RpcKind.RPC_BUILTIN, RpcConstants.DUMMY_CLIENT_ID);
+    }
+
     Call(Call call) {
       this(call.callId, call.retryCount, call.rpcKind, call.clientId,
           call.traceScope, call.callerContext);
@@ -702,6 +713,7 @@ public abstract class Server {
       return "Call#" + callId + " Retry#" + retryCount;
     }
 
+    @Override
     public Void run() throws Exception {
       return null;
     }
@@ -717,6 +729,10 @@ public abstract class Server {
       return (addr != null) ? addr.getHostAddress() : null;
     }
 
+    public String getProtocol() {
+      return null;
+    }
+
     /**
      * Allow a IPC response to be postponed instead of sent immediately
      * after the handler returns from the proxy method.  The intended use
@@ -798,6 +814,11 @@ public abstract class Server {
       this.rpcRequest = param;
     }
 
+    @Override
+    public String getProtocol() {
+      return "rpc";
+    }
+
     @Override
     public UserGroupInformation getRemoteUser() {
       return connection.user;
@@ -956,10 +977,16 @@ public abstract class Server {
             while (iter.hasNext()) {
               key = iter.next();
               iter.remove();
-              if (key.isValid()) {
+              try {
                 if (key.isReadable()) {
                   doRead(key);
                 }
+              } catch (CancelledKeyException cke) {
+                // something else closed the connection, ex. responder or
+                // the listener doing an idle scan.  ignore it and let them
+                // clean up.
+                LOG.info(Thread.currentThread().getName() +
+                    ": connection aborted from " + key.attachment());
               }
               key = null;
             }
@@ -969,6 +996,9 @@ public abstract class Server {
             }
           } catch (IOException ex) {
             LOG.error("Error in Reader", ex);
+          } catch (Throwable re) {
+            LOG.fatal("Bug in read selector!", re);
+            ExitUtil.terminate(1, "Bug in read selector!");
           }
         }
       }
@@ -1187,8 +1217,17 @@ public abstract class Server {
             SelectionKey key = iter.next();
             iter.remove();
             try {
-              if (key.isValid() && key.isWritable()) {
-                  doAsyncWrite(key);
+              if (key.isWritable()) {
+                doAsyncWrite(key);
+              }
+            } catch (CancelledKeyException cke) {
+              // something else closed the connection, ex. reader or the
+              // listener doing an idle scan.  ignore it and let them clean
+              // up
+              RpcCall call = (RpcCall)key.attachment();
+              if (call != null) {
+                LOG.info(Thread.currentThread().getName() +
+                    ": connection aborted from " + call.connection);
               }
             } catch (IOException e) {
               LOG.info(Thread.currentThread().getName() + ": doAsyncWrite threw exception " + e);
@@ -2314,33 +2353,15 @@ public abstract class Server {
       // Save the priority level assignment by the scheduler
       call.setPriorityLevel(callQueue.getPriorityLevel(call));
 
-      if (callQueue.isClientBackoffEnabled()) {
-        // if RPC queue is full, we will ask the RPC client to back off by
-        // throwing RetriableException. Whether RPC client will honor
-        // RetriableException and retry depends on client ipc retry policy.
-        // For example, FailoverOnNetworkExceptionRetry handles
-        // RetriableException.
-        queueRequestOrAskClientToBackOff(call);
-      } else {
-        callQueue.put(call);              // queue the call; maybe blocked here
+      try {
+        queueCall(call);
+      } catch (IOException ioe) {
+        throw new WrappedRpcServerException(
+            RpcErrorCodeProto.ERROR_RPC_SERVER, ioe);
       }
       incRpcCount();  // Increment the rpc count
     }
 
-    private void queueRequestOrAskClientToBackOff(Call call)
-        throws WrappedRpcServerException, InterruptedException {
-      // If rpc scheduler indicates back off based on performance
-      // degradation such as response time or rpc queue is full,
-      // we will ask the client to back off.
-      if (callQueue.shouldBackOff(call) || !callQueue.offer(call)) {
-        rpcMetrics.incrClientBackoff();
-        RetriableException retriableException =
-            new RetriableException("Server is too busy.");
-        throw new WrappedRpcServerExceptionSuppressed(
-            RpcErrorCodeProto.ERROR_RPC_SERVER, retriableException);
-      }
-    }
-
     /**
      * Establish RPC connection setup by negotiating SASL if required, then
      * reading and authorizing the connection header
@@ -2468,6 +2489,21 @@ public abstract class Server {
     }
   }
 
+  public void queueCall(Call call) throws IOException, InterruptedException {
+    if (!callQueue.isClientBackoffEnabled()) {
+      callQueue.put(call); // queue the call; maybe blocked here
+    } else if (callQueue.shouldBackOff(call) || !callQueue.offer(call)) {
+      // If rpc scheduler indicates back off based on performance degradation
+      // such as response time or rpc queue is full, we will ask the client
+      // to back off by throwing RetriableException. Whether the client will
+      // honor RetriableException and retry depends the client and its policy.
+      // For example, IPC clients using FailoverOnNetworkExceptionRetry handle
+      // RetriableException.
+      rpcMetrics.incrClientBackoff();
+      throw new RetriableException("Server is too busy.");
+    }
+  }
+
   /** Handles queued calls . */
   private class Handler extends Thread {
     public Handler(int instanceNumber) {

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -813,7 +813,7 @@ public class NetworkTopology {
       }
     }
     if (numOfDatanodes == 0) {
-      LOG.warn("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\").",
+      LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\").",
           String.valueOf(scope), String.valueOf(excludedScope));
       return null;
     }

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocksSocketFactory.java

@@ -31,6 +31,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SOCKS_SERVER_KEY;
+
 /**
  * Specialized SocketFactory to create sockets with a SOCKS proxy
  */
@@ -133,7 +135,7 @@ public class SocksSocketFactory extends SocketFactory implements
   @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
-    String proxyStr = conf.get("hadoop.socks.server");
+    String proxyStr = conf.get(HADOOP_SOCKS_SERVER_KEY);
     if ((proxyStr != null) && (proxyStr.length() > 0)) {
       setProxy(proxyStr);
     }

+ 2 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -104,12 +104,8 @@ public class Credentials implements Writable {
       for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
           tokenMap.entrySet()) {
         Token<? extends TokenIdentifier> token = e.getValue();
-        if (token instanceof Token.PrivateToken &&
-            ((Token.PrivateToken) token).getPublicService().equals(alias)) {
-          Token<? extends TokenIdentifier> privateToken =
-              new Token.PrivateToken<>(t);
-          privateToken.setService(token.getService());
-          tokensToAdd.put(e.getKey(), privateToken);
+        if (token.isPrivateCloneOf(alias)) {
+          tokensToAdd.put(e.getKey(), t.privateClone(token.getService()));
         }
       }
       tokenMap.putAll(tokensToAdd);

+ 118 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosAuthException.java

@@ -0,0 +1,118 @@
+/**
+ * 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.security;
+
+import static org.apache.hadoop.security.UGIExceptionMessages.*;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when {@link UserGroupInformation} failed with an unrecoverable error,
+ * such as failure in kerberos login/logout, invalid subject etc.
+ *
+ * Caller should not retry when catching this exception.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class KerberosAuthException extends IOException {
+  static final long serialVersionUID = 31L;
+
+  private String user;
+  private String principal;
+  private String keytabFile;
+  private String ticketCacheFile;
+  private String initialMessage;
+
+  public KerberosAuthException(String msg) {
+    super(msg);
+  }
+
+  public KerberosAuthException(Throwable cause) {
+    super(cause);
+  }
+
+  public KerberosAuthException(String initialMsg, Throwable cause) {
+    this(cause);
+    initialMessage = initialMsg;
+  }
+
+  public void setUser(final String u) {
+    user = u;
+  }
+
+  public void setPrincipal(final String p) {
+    principal = p;
+  }
+
+  public void setKeytabFile(final String k) {
+    keytabFile = k;
+  }
+
+  public void setTicketCacheFile(final String t) {
+    ticketCacheFile = t;
+  }
+
+  /** @return The initial message, or null if not set. */
+  public String getInitialMessage() {
+    return initialMessage;
+  }
+
+  /** @return The keytab file path, or null if not set. */
+  public String getKeytabFile() {
+    return keytabFile;
+  }
+
+  /** @return The principal, or null if not set. */
+  public String getPrincipal() {
+    return principal;
+  }
+
+  /** @return The ticket cache file path, or null if not set. */
+  public String getTicketCacheFile() {
+    return ticketCacheFile;
+  }
+
+  /** @return The user, or null if not set. */
+  public String getUser() {
+    return user;
+  }
+
+  @Override
+  public String getMessage() {
+    final StringBuilder sb = new StringBuilder();
+    if (initialMessage != null) {
+      sb.append(initialMessage);
+    }
+    if (user != null) {
+      sb.append(FOR_USER + user);
+    }
+    if (principal != null) {
+      sb.append(FOR_PRINCIPAL + principal);
+    }
+    if (keytabFile != null) {
+      sb.append(FROM_KEYTAB + keytabFile);
+    }
+    if (ticketCacheFile != null) {
+      sb.append(USING_TICKET_CACHE_FILE+ ticketCacheFile);
+    }
+    sb.append(" " + super.getMessage());
+    return sb.toString();
+  }
+}

+ 46 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UGIExceptionMessages.java

@@ -0,0 +1,46 @@
+/*
+ * 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.security;
+
+/**
+ * Standard strings to use in exception messages
+ * in {@link KerberosAuthException} when throwing.
+ */
+final class UGIExceptionMessages {
+
+  public static final String FAILURE_TO_LOGIN = "failure to login:";
+  public static final String FOR_USER = " for user: ";
+  public static final String FOR_PRINCIPAL = " for principal: ";
+  public static final String FROM_KEYTAB = " from keytab ";
+  public static final String LOGIN_FAILURE = "Login failure";
+  public static final String LOGOUT_FAILURE = "Logout failure";
+  public static final String MUST_FIRST_LOGIN =
+      "login must be done first";
+  public static final String MUST_FIRST_LOGIN_FROM_KEYTAB =
+      "loginUserFromKeyTab must be done first";
+  public static final String SUBJECT_MUST_CONTAIN_PRINCIPAL =
+      "Provided Subject must contain a KerberosPrincipal";
+  public static final String SUBJECT_MUST_NOT_BE_NULL =
+      "Subject must not be null";
+  public static final String USING_TICKET_CACHE_FILE =
+      " using ticket cache file: ";
+
+  //checkstyle: Utility classes should not have a public or default constructor.
+  private UGIExceptionMessages() {
+  }
+}

+ 47 - 58
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -21,6 +21,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_MET
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
+import static org.apache.hadoop.security.UGIExceptionMessages.*;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 import java.io.File;
@@ -38,7 +39,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -652,33 +652,7 @@ public class UserGroupInformation {
     }
     this.isKrbTkt = KerberosUtil.hasKerberosTicket(subject);
   }
-
-  /**
-   * Copies the Subject of this UGI and creates a new UGI with the new subject.
-   * This can be used to add credentials (e.g. tokens) to different copies of
-   * the same UGI, allowing multiple users with different tokens to reuse the
-   * UGI without re-authenticating with Kerberos.
-   * @return clone of the UGI with a new subject.
-   */
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  public UserGroupInformation copySubjectAndUgi() {
-    Subject subj = getSubject();
-    // The ctor will set other fields automatically from the principals.
-    return new UserGroupInformation(new Subject(false, subj.getPrincipals(),
-        cloneCredentials(subj.getPublicCredentials()),
-        cloneCredentials(subj.getPrivateCredentials())));
-  }
-
-  private static Set<Object> cloneCredentials(Set<Object> old) {
-    Set<Object> set = new HashSet<>();
-    // Make sure Hadoop credentials objects do not reuse the maps.
-    for (Object o : old) {
-      set.add(o instanceof Credentials ? new Credentials((Credentials)o) : o);
-    }
-    return set;
-  }
-
+  
   /**
    * checks if logged in using kerberos
    * @return true if the subject logged via keytab or has a Kerberos TGT
@@ -782,8 +756,11 @@ public class UserGroupInformation {
       ugi.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
       return ugi;
     } catch (LoginException le) {
-      throw new IOException("failure to login using ticket cache file " +
-          ticketCache, le);
+      KerberosAuthException kae =
+          new KerberosAuthException(FAILURE_TO_LOGIN, le);
+      kae.setUser(user);
+      kae.setTicketCacheFile(ticketCache);
+      throw kae;
     }
   }
 
@@ -792,16 +769,17 @@ public class UserGroupInformation {
    *
    * @param subject             The KerberosPrincipal to use in UGI
    *
-   * @throws IOException        if the kerberos login fails
+   * @throws IOException
+   * @throws KerberosAuthException if the kerberos login fails
    */
   public static UserGroupInformation getUGIFromSubject(Subject subject)
       throws IOException {
     if (subject == null) {
-      throw new IOException("Subject must not be null");
+      throw new KerberosAuthException(SUBJECT_MUST_NOT_BE_NULL);
     }
 
     if (subject.getPrincipals(KerberosPrincipal.class).isEmpty()) {
-      throw new IOException("Provided Subject must contain a KerberosPrincipal");
+      throw new KerberosAuthException(SUBJECT_MUST_CONTAIN_PRINCIPAL);
     }
 
     KerberosPrincipal principal =
@@ -921,7 +899,7 @@ public class UserGroupInformation {
       loginUser.spawnAutoRenewalThreadForUserCreds();
     } catch (LoginException le) {
       LOG.debug("failure to login", le);
-      throw new IOException("failure to login: " + le, le);
+      throw new KerberosAuthException(FAILURE_TO_LOGIN, le);
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("UGI loginUser:"+loginUser);
@@ -1028,7 +1006,8 @@ public class UserGroupInformation {
    * file and logs them in. They become the currently logged-in user.
    * @param user the principal name to load from the keytab
    * @param path the path to the keytab file
-   * @throws IOException if the keytab file can't be read
+   * @throws IOException
+   * @throws KerberosAuthException if it's a kerberos login exception.
    */
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
@@ -1057,8 +1036,10 @@ public class UserGroupInformation {
       if (start > 0) {
         metrics.loginFailure.add(Time.now() - start);
       }
-      throw new IOException("Login failure for " + user + " from keytab " + 
-                            path+ ": " + le, le);
+      KerberosAuthException kae = new KerberosAuthException(LOGIN_FAILURE, le);
+      kae.setUser(user);
+      kae.setKeytabFile(path);
+      throw kae;
     }
     LOG.info("Login successful for user " + keytabPrincipal
         + " using keytab file " + keytabFile);
@@ -1069,8 +1050,9 @@ public class UserGroupInformation {
    * This method assumes that the user logged in by calling
    * {@link #loginUserFromKeytab(String, String)}.
    *
-   * @throws IOException if a failure occurred in logout, or if the user did
-   * not log in by invoking loginUserFromKeyTab() before.
+   * @throws IOException
+   * @throws KerberosAuthException if a failure occurred in logout,
+   * or if the user did not log in by invoking loginUserFromKeyTab() before.
    */
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
@@ -1081,7 +1063,7 @@ public class UserGroupInformation {
     }
     LoginContext login = getLogin();
     if (login == null || keytabFile == null) {
-      throw new IOException("loginUserFromKeytab must be done first");
+      throw new KerberosAuthException(MUST_FIRST_LOGIN_FROM_KEYTAB);
     }
 
     try {
@@ -1092,9 +1074,10 @@ public class UserGroupInformation {
         login.logout();
       }
     } catch (LoginException le) {
-      throw new IOException("Logout failure for " + user + " from keytab " +
-          keytabFile + ": " + le,
-          le);
+      KerberosAuthException kae = new KerberosAuthException(LOGOUT_FAILURE, le);
+      kae.setUser(user.toString());
+      kae.setKeytabFile(keytabFile);
+      throw kae;
     }
 
     LOG.info("Logout successful for user " + keytabPrincipal
@@ -1105,6 +1088,7 @@ public class UserGroupInformation {
    * Re-login a user from keytab if TGT is expired or is close to expiry.
    * 
    * @throws IOException
+   * @throws KerberosAuthException if it's a kerberos login exception.
    */
   public synchronized void checkTGTAndReloginFromKeytab() throws IOException {
     if (!isSecurityEnabled()
@@ -1126,12 +1110,12 @@ public class UserGroupInformation {
    * happened already.
    * The Subject field of this UserGroupInformation object is updated to have
    * the new credentials.
-   * @throws IOException on a failure
+   * @throws IOException
+   * @throws KerberosAuthException on a failure
    */
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
-  public synchronized void reloginFromKeytab()
-  throws IOException {
+  public synchronized void reloginFromKeytab() throws IOException {
     if (!isSecurityEnabled() ||
          user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
          !isKeytab)
@@ -1151,7 +1135,7 @@ public class UserGroupInformation {
     
     LoginContext login = getLogin();
     if (login == null || keytabFile == null) {
-      throw new IOException("loginUserFromKeyTab must be done first");
+      throw new KerberosAuthException(MUST_FIRST_LOGIN_FROM_KEYTAB);
     }
     
     long start = 0;
@@ -1183,8 +1167,10 @@ public class UserGroupInformation {
       if (start > 0) {
         metrics.loginFailure.add(Time.now() - start);
       }
-      throw new IOException("Login failure for " + keytabPrincipal + 
-          " from keytab " + keytabFile + ": " + le, le);
+      KerberosAuthException kae = new KerberosAuthException(LOGIN_FAILURE, le);
+      kae.setPrincipal(keytabPrincipal);
+      kae.setKeytabFile(keytabFile);
+      throw kae;
     } 
   }
 
@@ -1193,19 +1179,19 @@ public class UserGroupInformation {
    * method assumes that login had happened already.
    * The Subject field of this UserGroupInformation object is updated to have
    * the new credentials.
-   * @throws IOException on a failure
+   * @throws IOException
+   * @throws KerberosAuthException on a failure
    */
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
-  public synchronized void reloginFromTicketCache()
-  throws IOException {
+  public synchronized void reloginFromTicketCache() throws IOException {
     if (!isSecurityEnabled() || 
         user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
         !isKrbTkt)
       return;
     LoginContext login = getLogin();
     if (login == null) {
-      throw new IOException("login must be done first");
+      throw new KerberosAuthException(MUST_FIRST_LOGIN);
     }
     long now = Time.now();
     if (!hasSufficientTimeElapsed(now)) {
@@ -1232,8 +1218,9 @@ public class UserGroupInformation {
       login.login();
       setLogin(login);
     } catch (LoginException le) {
-      throw new IOException("Login failure for " + getUserName() + ": " + le,
-          le);
+      KerberosAuthException kae = new KerberosAuthException(LOGIN_FAILURE, le);
+      kae.setUser(getUserName());
+      throw kae;
     } 
   }
 
@@ -1279,8 +1266,10 @@ public class UserGroupInformation {
       if (start > 0) {
         metrics.loginFailure.add(Time.now() - start);
       }
-      throw new IOException("Login failure for " + user + " from keytab " + 
-                            path + ": " + le, le);
+      KerberosAuthException kae = new KerberosAuthException(LOGIN_FAILURE, le);
+      kae.setUser(user);
+      kae.setKeytabFile(path);
+      throw kae;
     } finally {
       if(oldKeytabFile != null) keytabFile = oldKeytabFile;
       if(oldKeytabPrincipal != null) keytabPrincipal = oldKeytabPrincipal;
@@ -1611,7 +1600,7 @@ public class UserGroupInformation {
       Credentials creds = new Credentials(getCredentialsInternal());
       Iterator<Token<?>> iter = creds.getAllTokens().iterator();
       while (iter.hasNext()) {
-        if (iter.next() instanceof Token.PrivateToken) {
+        if (iter.next().isPrivate()) {
           iter.remove();
         }
       }

+ 52 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java

@@ -222,23 +222,67 @@ public class Token<T extends TokenIdentifier> implements Writable {
     service = newService;
   }
 
+  /**
+   * Whether this is a private token.
+   * @return false always for non-private tokens
+   */
+  public boolean isPrivate() {
+    return false;
+  }
+
+  /**
+   * Whether this is a private clone of a public token.
+   * @param thePublicService the public service name
+   * @return false always for non-private tokens
+   */
+  public boolean isPrivateCloneOf(Text thePublicService) {
+    return false;
+  }
+
+  /**
+   * Create a private clone of a public token.
+   * @param newService the new service name
+   * @return a private token
+   */
+  public Token<T> privateClone(Text newService) {
+    return new PrivateToken<>(this, newService);
+  }
+
   /**
    * Indicates whether the token is a clone.  Used by HA failover proxy
    * to indicate a token should not be visible to the user via
    * UGI.getCredentials()
    */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static class PrivateToken<T extends TokenIdentifier> extends Token<T> {
+  static class PrivateToken<T extends TokenIdentifier> extends Token<T> {
     final private Text publicService;
 
-    public PrivateToken(Token<T> token) {
-      super(token);
-      publicService = new Text(token.getService());
+    PrivateToken(Token<T> publicToken, Text newService) {
+      super(publicToken.identifier, publicToken.password, publicToken.kind,
+          newService);
+      assert !publicToken.isPrivate();
+      publicService = publicToken.service;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Cloned private token " + this + " from " + publicToken);
+      }
     }
 
-    public Text getPublicService() {
-      return publicService;
+    /**
+     * Whether this is a private token.
+     * @return true always for private tokens
+     */
+    @Override
+    public boolean isPrivate() {
+      return true;
+    }
+
+    /**
+     * Whether this is a private clone of a public token.
+     * @param thePublicService the public service name
+     * @return true when the public service is the same as specified
+     */
+    @Override
+    public boolean isPrivateCloneOf(Text thePublicService) {
+      return publicService.equals(thePublicService);
     }
 
     @Override

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
  * A class that provides a line reader from an input stream.
  * Depending on the constructor used, lines will either be terminated by:
@@ -89,7 +91,7 @@ public class LineReader implements Closeable {
    * @throws IOException
    */
   public LineReader(InputStream in, Configuration conf) throws IOException {
-    this(in, conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE));
+    this(in, conf.getInt(IO_FILE_BUFFER_SIZE_KEY, DEFAULT_BUFFER_SIZE));
   }
 
   /**
@@ -136,7 +138,7 @@ public class LineReader implements Closeable {
   public LineReader(InputStream in, Configuration conf,
       byte[] recordDelimiterBytes) throws IOException {
     this.in = in;
-    this.bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
+    this.bufferSize = conf.getInt(IO_FILE_BUFFER_SIZE_KEY, DEFAULT_BUFFER_SIZE);
     this.buffer = new byte[this.bufferSize];
     this.recordDelimiterBytes = recordDelimiterBytes;
   }

+ 31 - 27
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java

@@ -100,36 +100,40 @@ public class SysInfoWindows extends SysInfo {
       String sysInfoStr = getSystemInfoInfoFromShell();
       if (sysInfoStr != null) {
         final int sysInfoSplitCount = 11;
-        String[] sysInfo = sysInfoStr.substring(0, sysInfoStr.indexOf("\r\n"))
-            .split(",");
-        if (sysInfo.length == sysInfoSplitCount) {
-          try {
-            vmemSize = Long.parseLong(sysInfo[0]);
-            memSize = Long.parseLong(sysInfo[1]);
-            vmemAvailable = Long.parseLong(sysInfo[2]);
-            memAvailable = Long.parseLong(sysInfo[3]);
-            numProcessors = Integer.parseInt(sysInfo[4]);
-            cpuFrequencyKhz = Long.parseLong(sysInfo[5]);
-            cumulativeCpuTimeMs = Long.parseLong(sysInfo[6]);
-            storageBytesRead = Long.parseLong(sysInfo[7]);
-            storageBytesWritten = Long.parseLong(sysInfo[8]);
-            netBytesRead = Long.parseLong(sysInfo[9]);
-            netBytesWritten = Long.parseLong(sysInfo[10]);
-            if (lastCumCpuTimeMs != -1) {
-              /**
-               * This number will be the aggregated usage across all cores in
-               * [0.0, 100.0]. For example, it will be 400.0 if there are 8
-               * cores and each of them is running at 50% utilization.
-               */
-              cpuUsage = (cumulativeCpuTimeMs - lastCumCpuTimeMs)
-                  * 100F / refreshInterval;
+        int index = sysInfoStr.indexOf("\r\n");
+        if (index >= 0) {
+          String[] sysInfo = sysInfoStr.substring(0, index).split(",");
+          if (sysInfo.length == sysInfoSplitCount) {
+            try {
+              vmemSize = Long.parseLong(sysInfo[0]);
+              memSize = Long.parseLong(sysInfo[1]);
+              vmemAvailable = Long.parseLong(sysInfo[2]);
+              memAvailable = Long.parseLong(sysInfo[3]);
+              numProcessors = Integer.parseInt(sysInfo[4]);
+              cpuFrequencyKhz = Long.parseLong(sysInfo[5]);
+              cumulativeCpuTimeMs = Long.parseLong(sysInfo[6]);
+              storageBytesRead = Long.parseLong(sysInfo[7]);
+              storageBytesWritten = Long.parseLong(sysInfo[8]);
+              netBytesRead = Long.parseLong(sysInfo[9]);
+              netBytesWritten = Long.parseLong(sysInfo[10]);
+              if (lastCumCpuTimeMs != -1) {
+                /**
+                 * This number will be the aggregated usage across all cores in
+                 * [0.0, 100.0]. For example, it will be 400.0 if there are 8
+                 * cores and each of them is running at 50% utilization.
+                 */
+                cpuUsage = (cumulativeCpuTimeMs - lastCumCpuTimeMs)
+                    * 100F / refreshInterval;
+              }
+            } catch (NumberFormatException nfe) {
+              LOG.warn("Error parsing sysInfo", nfe);
             }
-          } catch (NumberFormatException nfe) {
-            LOG.warn("Error parsing sysInfo", nfe);
+          } else {
+            LOG.warn("Expected split length of sysInfo to be "
+                + sysInfoSplitCount + ". Got " + sysInfo.length);
           }
         } else {
-          LOG.warn("Expected split length of sysInfo to be "
-              + sysInfoSplitCount + ". Got " + sysInfo.length);
+          LOG.warn("Wrong output from sysInfo: " + sysInfoStr);
         }
       }
     }

+ 5 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/Hash.java

@@ -22,6 +22,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_UTIL_HASH_TYPE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_UTIL_HASH_TYPE_KEY;
+
 /**
  * This class represents a common API for hashing functions.
  */
@@ -59,7 +62,8 @@ public abstract class Hash {
    * @return one of the predefined constants
    */
   public static int getHashType(Configuration conf) {
-    String name = conf.get("hadoop.util.hash.type", "murmur");
+    String name = conf.get(HADOOP_UTIL_HASH_TYPE_KEY,
+        HADOOP_UTIL_HASH_TYPE_DEFAULT);
     return parseHashType(name);
   }
   

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -93,7 +93,7 @@
   <description>
     The name of the Network Interface from which the service should determine
     its host name for Kerberos login. e.g. eth2. In a multi-homed environment,
-    the setting can be used to affect the _HOST subsitution in the service
+    the setting can be used to affect the _HOST substitution in the service
     Kerberos principal. If this configuration value is not set, the service
     will use its default hostname as returned by
     InetAddress.getLocalHost().getCanonicalHostName().
@@ -400,7 +400,7 @@
     The number of levels to go up the group hierarchy when determining
     which groups a user is part of. 0 Will represent checking just the
     group that the user belongs to.  Each additional level will raise the
-    time it takes to exectue a query by at most
+    time it takes to execute a query by at most
     hadoop.security.group.mapping.ldap.directory.search.timeout.
     The default will usually be appropriate for all LDAP systems.
   </description>
@@ -1939,7 +1939,7 @@
   <value>dr.who=;</value>
   <description>
     Static mapping of user to groups. This will override the groups if
-    available in the system for the specified user. In otherwords, groups
+    available in the system for the specified user. In other words, groups
     look-up will not happen for these users, instead groups mapped in this
     configuration will be used.
     Mapping should be in this format.

+ 2 - 1
hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md

@@ -504,7 +504,7 @@ See [HDFS Snapshots Guide](../hadoop-hdfs/HdfsSnapshots.html).
 rm
 ----
 
-Usage: `hadoop fs -rm [-f] [-r |-R] [-skipTrash] URI [URI ...]`
+Usage: `hadoop fs -rm [-f] [-r |-R] [-skipTrash] [-safely] URI [URI ...]`
 
 Delete files specified as args.
 
@@ -523,6 +523,7 @@ Options:
 * The -R option deletes the directory and any content under it recursively.
 * The -r option is equivalent to -R.
 * The -skipTrash option will bypass trash, if enabled, and delete the specified file(s) immediately. This can be useful when it is necessary to delete files from an over-quota directory.
+* The -safely option will require safety confirmation before deleting directory with total number of files greater than `hadoop.shell.delete.limit.num.files` (in core-site.xml, default: 100). It can be used with -skipTrash to prevent accidental deletion of large directories. Delay is expected when walking over large directory recursively to count the number of files to be deleted before the confirmation.
 
 Example:
 

+ 58 - 19
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -669,19 +669,40 @@ exists in the metadata, but no copies of any its blocks can be located;
 
 ### `boolean delete(Path p, boolean recursive)`
 
+Delete a path, be it a file, symbolic link or directory. The
+`recursive` flag indicates whether a recursive delete should take place —if
+unset then a non-empty directory cannot be deleted.
+
+Except in the special case of the root directory, if this API call
+completed successfully then there is nothing at the end of the path.
+That is: the outcome is desired. The return flag simply tells the caller
+whether or not any change was made to the state of the filesystem.
+
+*Note*: many uses of this method surround it with checks for the return value being
+false, raising exception if so. For example
+
+```java
+if (!fs.delete(path, true)) throw new IOException("Could not delete " + path);
+```
+
+This pattern is not needed. Code SHOULD just call `delete(path, recursive)` and
+assume the destination is no longer present —except in the special case of root
+directories, which will always remain (see below for special coverage of root directories).
+
 #### Preconditions
 
-A directory with children and recursive == false cannot be deleted
+A directory with children and `recursive == False` cannot be deleted
 
     if isDir(FS, p) and not recursive and (children(FS, p) != {}) : raise IOException
 
+(HDFS raises `PathIsNotEmptyDirectoryException` here.)
 
 #### Postconditions
 
 
 ##### Nonexistent path
 
-If the file does not exist the FS state does not change
+If the file does not exist the filesystem state does not change
 
     if not exists(FS, p):
         FS' = FS
@@ -700,7 +721,7 @@ A path referring to a file is removed, return value: `True`
         result = True
 
 
-##### Empty root directory
+##### Empty root directory, `recursive == False`
 
 Deleting an empty root does not change the filesystem state
 and may return true or false.
@@ -711,7 +732,10 @@ and may return true or false.
 
 There is no consistent return code from an attempt to delete the root directory.
 
-##### Empty (non-root) directory
+Implementations SHOULD return true; this avoids code which checks for a false
+return value from overreacting.
+
+##### Empty (non-root) directory `recursive == False`
 
 Deleting an empty directory that is not root will remove the path from the FS and
 return true.
@@ -721,26 +745,41 @@ return true.
         result = True
 
 
-##### Recursive delete of root directory
+##### Recursive delete of non-empty root directory
 
 Deleting a root path with children and `recursive==True`
  can do one of two things.
 
-The POSIX model assumes that if the user has
+1. The POSIX model assumes that if the user has
 the correct permissions to delete everything,
 they are free to do so (resulting in an empty filesystem).
 
-    if isDir(FS, p) and isRoot(p) and recursive :
-        FS' = ({["/"]}, {}, {}, {})
-        result = True
+        if isDir(FS, p) and isRoot(p) and recursive :
+            FS' = ({["/"]}, {}, {}, {})
+            result = True
 
-In contrast, HDFS never permits the deletion of the root of a filesystem; the
-filesystem can be taken offline and reformatted if an empty
+1. HDFS never permits the deletion of the root of a filesystem; the
+filesystem must be taken offline and reformatted if an empty
 filesystem is desired.
 
-    if isDir(FS, p) and isRoot(p) and recursive :
-        FS' = FS
-        result = False
+        if isDir(FS, p) and isRoot(p) and recursive :
+            FS' = FS
+            result = False
+
+HDFS has the notion of *Protected Directories*, which are declared in
+the option `fs.protected.directories`. Any attempt to delete such a directory
+or a parent thereof raises an `AccessControlException`. Accordingly, any
+attempt to delete the root directory SHALL, if there is a protected directory,
+result in such an exception being raised.
+
+This specification does not recommend any specific action. Do note, however,
+that the POSIX model assumes that there is a permissions model such that normal
+users do not have the permission to delete that root directory; it is an action
+which only system administrators should be able to perform.
+
+Any filesystem client which interacts with a remote filesystem which lacks
+such a security model, MAY reject calls to `delete("/", true)` on the basis
+that it makes it too easy to lose data.
 
 ##### Recursive delete of non-root directory
 
@@ -766,11 +805,11 @@ removes the path and all descendants
 
 #### Implementation Notes
 
-* S3N, Swift, FTP and potentially other non-traditional FileSystems
-implement `delete()` as recursive listing and file delete operation.
-This can break the expectations of client applications -and means that
-they cannot be used as drop-in replacements for HDFS.
-
+* Object Stores and other non-traditional filesystems onto which a directory
+ tree is emulated, tend to implement `delete()` as recursive listing and
+entry-by-entry delete operation.
+This can break the expectations of client applications for O(1) atomic directory
+deletion, preventing the stores' use as drop-in replacements for HDFS.
 
 ### `boolean rename(Path src, Path d)`
 

+ 119 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java

@@ -18,11 +18,15 @@
 package org.apache.hadoop.conf;
 
 import java.io.StringWriter;
+import java.io.PrintWriter;
 import java.io.StringReader;
 import java.util.HashMap;
 import java.util.Map;
 
 import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.ServletConfig;
+import javax.servlet.ServletContext;
 import javax.ws.rs.core.HttpHeaders;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -34,17 +38,36 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
-import junit.framework.TestCase;
+import com.google.common.base.Strings;
+
+import org.apache.hadoop.http.HttpServer2;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.mock;
+import static org.junit.Assert.*;
 
 /**
  * Basic test case that the ConfServlet can write configuration
  * to its output in XML and JSON format.
  */
-public class TestConfServlet extends TestCase {
+public class TestConfServlet {
   private static final String TEST_KEY = "testconfservlet.key";
   private static final String TEST_VAL = "testval";
+  private static final Map<String, String> TEST_PROPERTIES =
+      new HashMap<String, String>();
+  private static final Map<String, String> TEST_FORMATS =
+      new HashMap<String, String>();
+
+  @BeforeClass
+  public static void initTestProperties() {
+    TEST_PROPERTIES.put("test.key1", "value1");
+    TEST_PROPERTIES.put("test.key2", "value2");
+    TEST_PROPERTIES.put("test.key3", "value3");
+    TEST_FORMATS.put(ConfServlet.FORMAT_XML, "application/xml");
+    TEST_FORMATS.put(ConfServlet.FORMAT_JSON, "application/json");
+  }
 
   private Configuration getTestConf() {
     Configuration testConf = new Configuration();
@@ -52,6 +75,14 @@ public class TestConfServlet extends TestCase {
     return testConf;
   }
 
+  private Configuration getMultiPropertiesConf() {
+    Configuration testConf = new Configuration(false);
+    for(String key : TEST_PROPERTIES.keySet()) {
+      testConf.set(key, TEST_PROPERTIES.get(key));
+    }
+    return testConf;
+  }
+
   @Test
   public void testParseHeaders() throws Exception {
     HashMap<String, String> verifyMap = new HashMap<String, String>();
@@ -71,6 +102,92 @@ public class TestConfServlet extends TestCase {
     }
   }
 
+  private void verifyGetProperty(Configuration conf, String format,
+      String propertyName) throws Exception {
+    StringWriter sw = null;
+    PrintWriter pw = null;
+    ConfServlet service = null;
+    try {
+      service = new ConfServlet();
+      ServletConfig servletConf = mock(ServletConfig.class);
+      ServletContext context = mock(ServletContext.class);
+      service.init(servletConf);
+      when(context.getAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE))
+        .thenReturn(conf);
+      when(service.getServletContext())
+        .thenReturn(context);
+
+      HttpServletRequest request = mock(HttpServletRequest.class);
+      when(request.getHeader(HttpHeaders.ACCEPT))
+        .thenReturn(TEST_FORMATS.get(format));
+      when(request.getParameter("name"))
+        .thenReturn(propertyName);
+
+      HttpServletResponse response = mock(HttpServletResponse.class);
+      sw = new StringWriter();
+      pw = new PrintWriter(sw);
+      when(response.getWriter()).thenReturn(pw);
+
+      // response request
+      service.doGet(request, response);
+      String result = sw.toString().trim();
+
+      // if property name is null or empty, expect all properties
+      // in the response
+      if (Strings.isNullOrEmpty(propertyName)) {
+        for(String key : TEST_PROPERTIES.keySet()) {
+          assertTrue(result.contains(key) &&
+              result.contains(TEST_PROPERTIES.get(key)));
+        }
+      } else {
+        if(conf.get(propertyName) != null) {
+          // if property name is not empty and property is found
+          assertTrue(result.contains(propertyName));
+          for(String key : TEST_PROPERTIES.keySet()) {
+            if(!key.equals(propertyName)) {
+              assertFalse(result.contains(key));
+            }
+          }
+        } else {
+          // if property name is not empty, and it's not in configuration
+          // expect proper error code and error message is set to the response
+          Mockito.verify(response).sendError(
+              Mockito.eq(HttpServletResponse.SC_NOT_FOUND),
+              Mockito.eq("Property " + propertyName + " not found"));
+        }
+      }
+    } finally {
+      if (sw != null) {
+        sw.close();
+      }
+      if (pw != null) {
+        pw.close();
+      }
+      if (service != null) {
+        service.destroy();
+      }
+    }
+  }
+
+  @Test
+  public void testGetProperty() throws Exception {
+    Configuration configurations = getMultiPropertiesConf();
+    // list various of property names
+    String[] testKeys = new String[] {
+        "test.key1",
+        "test.unknown.key",
+        "",
+        "test.key2",
+        null
+    };
+
+    for(String format : TEST_FORMATS.keySet()) {
+      for(String key : testKeys) {
+        verifyGetProperty(configurations, format, key);
+      }
+    }
+  }
+
   @Test
   @SuppressWarnings("unchecked")
   public void testWriteJson() throws Exception {
@@ -109,7 +226,6 @@ public class TestConfServlet extends TestCase {
     for (int i = 0; i < nameNodes.getLength(); i++) {
       Node nameNode = nameNodes.item(i);
       String key = nameNode.getTextContent();
-      System.err.println("xml key: " + key);
       if (TEST_KEY.equals(key)) {
         foundSetting = true;
         Element propertyElem = (Element)nameNode.getParentNode();

+ 136 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -42,7 +42,6 @@ import static java.util.concurrent.TimeUnit.*;
 
 import junit.framework.TestCase;
 import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.fail;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
@@ -1140,7 +1139,19 @@ public class TestConfiguration extends TestCase {
       this.properties = properties;
     }
   }
-  
+
+  static class SingleJsonConfiguration {
+    private JsonProperty property;
+
+    public JsonProperty getProperty() {
+      return property;
+    }
+
+    public void setProperty(JsonProperty property) {
+      this.property = property;
+    }
+  }
+
   static class JsonProperty {
     String key;
     public String getKey() {
@@ -1171,7 +1182,14 @@ public class TestConfiguration extends TestCase {
     boolean isFinal;
     String resource;
   }
-  
+
+  private Configuration getActualConf(String xmlStr) {
+    Configuration ac = new Configuration(false);
+    InputStream in = new ByteArrayInputStream(xmlStr.getBytes());
+    ac.addResource(in);
+    return ac;
+  }
+
   public void testGetSetTrimmedNames() throws IOException {
     Configuration conf = new Configuration(false);
     conf.set(" name", "value");
@@ -1180,7 +1198,121 @@ public class TestConfiguration extends TestCase {
     assertEquals("value", conf.getRaw("  name  "));
   }
 
-  public void testDumpConfiguration () throws IOException {
+  public void testDumpProperty() throws IOException {
+    StringWriter outWriter = new StringWriter();
+    ObjectMapper mapper = new ObjectMapper();
+    String jsonStr = null;
+    String xmlStr = null;
+    try {
+      Configuration testConf = new Configuration(false);
+      out = new BufferedWriter(new FileWriter(CONFIG));
+      startConfig();
+      appendProperty("test.key1", "value1");
+      appendProperty("test.key2", "value2", true);
+      appendProperty("test.key3", "value3");
+      endConfig();
+      Path fileResource = new Path(CONFIG);
+      testConf.addResource(fileResource);
+      out.close();
+
+      // case 1: dump an existing property
+      // test json format
+      outWriter = new StringWriter();
+      Configuration.dumpConfiguration(testConf, "test.key2", outWriter);
+      jsonStr = outWriter.toString();
+      outWriter.close();
+      mapper = new ObjectMapper();
+      SingleJsonConfiguration jconf1 =
+          mapper.readValue(jsonStr, SingleJsonConfiguration.class);
+      JsonProperty jp1 = jconf1.getProperty();
+      assertEquals("test.key2", jp1.getKey());
+      assertEquals("value2", jp1.getValue());
+      assertEquals(true, jp1.isFinal);
+      assertEquals(fileResource.toUri().getPath(), jp1.getResource());
+
+      // test xml format
+      outWriter = new StringWriter();
+      testConf.writeXml("test.key2", outWriter);
+      xmlStr = outWriter.toString();
+      outWriter.close();
+      Configuration actualConf1 = getActualConf(xmlStr);
+      assertEquals(1, actualConf1.size());
+      assertEquals("value2", actualConf1.get("test.key2"));
+      assertTrue(actualConf1.getFinalParameters().contains("test.key2"));
+      assertEquals(fileResource.toUri().getPath(),
+          actualConf1.getPropertySources("test.key2")[0]);
+
+      // case 2: dump an non existing property
+      // test json format
+      try {
+        outWriter = new StringWriter();
+        Configuration.dumpConfiguration(testConf,
+            "test.unknown.key", outWriter);
+        outWriter.close();
+      } catch (Exception e) {
+        assertTrue(e instanceof IllegalArgumentException);
+        assertTrue(e.getMessage().contains("test.unknown.key") &&
+            e.getMessage().contains("not found"));
+      }
+      // test xml format
+      try {
+        outWriter = new StringWriter();
+        testConf.writeXml("test.unknown.key", outWriter);
+        outWriter.close();
+      } catch (Exception e) {
+        assertTrue(e instanceof IllegalArgumentException);
+        assertTrue(e.getMessage().contains("test.unknown.key") &&
+            e.getMessage().contains("not found"));
+      }
+
+      // case 3: specify a null property, ensure all configurations are dumped
+      outWriter = new StringWriter();
+      Configuration.dumpConfiguration(testConf, null, outWriter);
+      jsonStr = outWriter.toString();
+      mapper = new ObjectMapper();
+      JsonConfiguration jconf3 =
+          mapper.readValue(jsonStr, JsonConfiguration.class);
+      assertEquals(3, jconf3.getProperties().length);
+
+      outWriter = new StringWriter();
+      testConf.writeXml(null, outWriter);
+      xmlStr = outWriter.toString();
+      outWriter.close();
+      Configuration actualConf3 = getActualConf(xmlStr);
+      assertEquals(3, actualConf3.size());
+      assertTrue(actualConf3.getProps().containsKey("test.key1") &&
+          actualConf3.getProps().containsKey("test.key2") &&
+          actualConf3.getProps().containsKey("test.key3"));
+
+      // case 4: specify an empty property, ensure all configurations are dumped
+      outWriter = new StringWriter();
+      Configuration.dumpConfiguration(testConf, "", outWriter);
+      jsonStr = outWriter.toString();
+      mapper = new ObjectMapper();
+      JsonConfiguration jconf4 =
+          mapper.readValue(jsonStr, JsonConfiguration.class);
+      assertEquals(3, jconf4.getProperties().length);
+
+      outWriter = new StringWriter();
+      testConf.writeXml("", outWriter);
+      xmlStr = outWriter.toString();
+      outWriter.close();
+      Configuration actualConf4 = getActualConf(xmlStr);
+      assertEquals(3, actualConf4.size());
+      assertTrue(actualConf4.getProps().containsKey("test.key1") &&
+          actualConf4.getProps().containsKey("test.key2") &&
+          actualConf4.getProps().containsKey("test.key3"));
+    } finally {
+      if(outWriter != null) {
+        outWriter.close();
+      }
+      if(out != null) {
+        out.close();
+      }
+    }
+  }
+
+  public void testDumpConfiguration() throws IOException {
     StringWriter outWriter = new StringWriter();
     Configuration.dumpConfiguration(conf, outWriter);
     String jsonStr = outWriter.toString();

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java

@@ -77,7 +77,9 @@ public abstract class FileContextURIBase {
   public void tearDown() throws Exception {
     // Clean up after test completion
     // No need to clean fc1 as fc1 and fc2 points same location
-    fc2.delete(BASE, true);
+    if (fc2 != null) {
+      fc2.delete(BASE, true);
+    }
   }
 
   @Test

+ 75 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java

@@ -0,0 +1,75 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.fs;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test to make sure df can run and work.
+ */
+public class TestDFCachingGetSpaceUsed {
+  final static private File DF_DIR = GenericTestUtils.getTestDir("testdfspace");
+  public static final int FILE_SIZE = 1024;
+
+  @Before
+  public void setUp() {
+    FileUtil.fullyDelete(DF_DIR);
+    assertTrue(DF_DIR.mkdirs());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    FileUtil.fullyDelete(DF_DIR);
+  }
+
+  @Test
+  public void testCanBuildRun() throws Exception {
+    File file = writeFile("testCanBuild");
+
+    GetSpaceUsed instance = new CachingGetSpaceUsed.Builder()
+        .setPath(file)
+        .setInterval(50060)
+        .setKlass(DFCachingGetSpaceUsed.class)
+        .build();
+    assertTrue(instance instanceof DFCachingGetSpaceUsed);
+    assertTrue(instance.getUsed() >= FILE_SIZE - 20);
+    ((DFCachingGetSpaceUsed) instance).close();
+  }
+
+  private File writeFile(String fileName) throws IOException {
+    File f = new File(DF_DIR, fileName);
+    assertTrue(f.createNewFile());
+    RandomAccessFile randomAccessFile = new RandomAccessFile(f, "rws");
+    randomAccessFile.writeUTF(RandomStringUtils.randomAlphabetic(FILE_SIZE));
+    randomAccessFile.getFD().sync();
+    randomAccessFile.close();
+    return f;
+  }
+
+}

+ 1 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDelegationTokenRenewer.java

@@ -49,7 +49,6 @@ public class TestDelegationTokenRenewer {
     renewer = DelegationTokenRenewer.getInstance();
   }
   
-  @SuppressWarnings("unchecked")
   @Test
   public void testAddRemoveRenewAction() throws IOException,
       InterruptedException {
@@ -81,7 +80,7 @@ public class TestDelegationTokenRenewer {
     verify(token).cancel(eq(conf));
 
     verify(fs, never()).getDelegationToken(null);
-    verify(fs, never()).setDelegationToken(any(Token.class));
+    verify(fs, never()).setDelegationToken(any());
     
     assertEquals("FileSystem not removed from DelegationTokenRenewer", 0,
         renewer.getRenewQueueLength());

+ 4 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemInitialization.java

@@ -47,16 +47,12 @@ public class TestFileSystemInitialization {
 
   @Test
   public void testMissingLibraries() {
-    boolean catched = false;
     try {
       Configuration conf = new Configuration();
-      FileSystem.getFileSystemClass("s3a", conf);
-    } catch (Exception e) {
-      catched = true;
-    } catch (ServiceConfigurationError e) {
-      // S3A shouldn't find AWS SDK and fail
-      catched = true;
+      Class<? extends FileSystem> fs = FileSystem.getFileSystemClass("s3a",
+          conf);
+      fail("Expected an exception, got a filesystem: " + fs);
+    } catch (Exception | ServiceConfigurationError expected) {
     }
-    assertTrue(catched);
   }
 }

+ 33 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java

@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.FileStatus;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.deleteChildren;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.listChildren;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.toList;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk;
 
@@ -62,12 +64,40 @@ public abstract class AbstractContractRootDirectoryTest extends AbstractFSContra
   }
 
   @Test
-  public void testRmEmptyRootDirNonRecursive() throws Throwable {
+  public void testRmEmptyRootDirRecursive() throws Throwable {
     //extra sanity checks here to avoid support calls about complete loss of data
     skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
     Path root = new Path("/");
     assertIsDirectory(root);
     boolean deleted = getFileSystem().delete(root, true);
+    LOG.info("rm -r / of empty dir result is {}", deleted);
+    assertIsDirectory(root);
+  }
+
+  @Test
+  public void testRmEmptyRootDirNonRecursive() throws Throwable {
+    // extra sanity checks here to avoid support calls about complete loss
+    // of data
+    skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
+    Path root = new Path("/");
+    assertIsDirectory(root);
+    // make sure it is clean
+    FileSystem fs = getFileSystem();
+    deleteChildren(fs, root, true);
+    FileStatus[] children = listChildren(fs, root);
+    if (children.length > 0) {
+      StringBuilder error = new StringBuilder();
+      error.append("Deletion of child entries failed, still have")
+          .append(children.length)
+          .append(System.lineSeparator());
+      for (FileStatus child : children) {
+        error.append("  ").append(child.getPath())
+            .append(System.lineSeparator());
+      }
+      fail(error.toString());
+    }
+    // then try to delete the empty one
+    boolean deleted = fs.delete(root, false);
     LOG.info("rm / of empty dir result is {}", deleted);
     assertIsDirectory(root);
   }
@@ -88,6 +118,8 @@ public abstract class AbstractContractRootDirectoryTest extends AbstractFSContra
     } catch (IOException e) {
       //expected
       handleExpectedException(e);
+      // and the file must still be present
+      assertIsFile(file);
     } finally {
       getFileSystem().delete(file, false);
     }

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java

@@ -359,7 +359,7 @@ public abstract class AbstractFSContractTestBase extends Assert
     assertEquals(text + " wrong read result " + result, -1, result);
   }
 
-  boolean rename(Path src, Path dst) throws IOException {
+  protected boolean rename(Path src, Path dst) throws IOException {
     return getFileSystem().rename(src, dst);
   }
 

+ 44 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -47,6 +47,9 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
  * Utilities used across test cases.
  */
@@ -55,8 +58,6 @@ public class ContractTestUtils extends Assert {
   private static final Logger LOG =
       LoggerFactory.getLogger(ContractTestUtils.class);
 
-  public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size";
-
   // For scale testing, we can repeatedly write small chunk data to generate
   // a large file.
   public static final String IO_CHUNK_BUFFER_SIZE = "io.chunk.buffer.size";
@@ -150,8 +151,8 @@ public class ContractTestUtils extends Assert {
     FSDataOutputStream out = fs.create(path,
                                        overwrite,
                                        fs.getConf()
-                                         .getInt(IO_FILE_BUFFER_SIZE,
-                                                 4096),
+                                         .getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                             IO_FILE_BUFFER_SIZE_DEFAULT),
                                        (short) 1,
                                        buffersize);
     out.write(src, 0, len);
@@ -392,6 +393,45 @@ public class ContractTestUtils extends Assert {
     rejectRootOperation(path, false);
   }
 
+  /**
+   * List then delete the children of a path, but not the path itself.
+   * This can be used to delete the entries under a root path when that
+   * FS does not support {@code delete("/")}.
+   * @param fileSystem filesystem
+   * @param path path to delete
+   * @param recursive flag to indicate child entry deletion should be recursive
+   * @return the number of child entries found and deleted (not including
+   * any recursive children of those entries)
+   * @throws IOException problem in the deletion process.
+   */
+  public static int deleteChildren(FileSystem fileSystem,
+      Path path,
+      boolean recursive) throws IOException {
+    FileStatus[] children = listChildren(fileSystem, path);
+    for (FileStatus entry : children) {
+      fileSystem.delete(entry.getPath(), recursive);
+    }
+    return children.length;
+  }
+
+  /**
+   * List all children of a path, but not the path itself in the case
+   * that the path refers to a file or empty directory.
+   * @param fileSystem FS
+   * @param path path
+   * @return a list of children, and never the path itself.
+   * @throws IOException problem in the list process
+   */
+  public static FileStatus[] listChildren(FileSystem fileSystem,
+      Path path) throws IOException {
+    FileStatus[] entries = fileSystem.listStatus(path);
+    if (entries.length == 1 && path.equals(entries[0].getPath())) {
+      // this is the path: ignore
+      return new FileStatus[]{};
+    } else {
+      return entries;
+    }
+  }
 
   public static void noteAction(String action) {
     if (LOG.isDebugEnabled()) {

+ 85 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -64,6 +64,7 @@ import java.net.ConnectException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -926,6 +927,90 @@ public class TestRPC extends TestRpcBase {
     }
   }
 
+  @Test(timeout=30000)
+  public void testExternalCall() throws Exception {
+    final UserGroupInformation ugi = UserGroupInformation
+        .createUserForTesting("user123", new String[0]);
+    final IOException expectedIOE = new IOException("boom");
+
+    // use 1 handler so the callq can be plugged
+    final Server server = setupTestServer(conf, 1);
+    try {
+      final AtomicBoolean result = new AtomicBoolean();
+
+      ExternalCall<String> remoteUserCall = newExtCall(ugi,
+          new PrivilegedExceptionAction<String>() {
+            @Override
+            public String run() throws Exception {
+              return UserGroupInformation.getCurrentUser().getUserName();
+            }
+          });
+
+      ExternalCall<String> exceptionCall = newExtCall(ugi,
+          new PrivilegedExceptionAction<String>() {
+            @Override
+            public String run() throws Exception {
+              throw expectedIOE;
+            }
+          });
+
+      final CountDownLatch latch = new CountDownLatch(1);
+      final CyclicBarrier barrier = new CyclicBarrier(2);
+
+      ExternalCall<Void> barrierCall = newExtCall(ugi,
+          new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+              // notify we are in a handler and then wait to keep the callq
+              // plugged up
+              latch.countDown();
+              barrier.await();
+              return null;
+            }
+          });
+
+      server.queueCall(barrierCall);
+      server.queueCall(exceptionCall);
+      server.queueCall(remoteUserCall);
+
+      // wait for barrier call to enter the handler, check that the other 2
+      // calls are actually queued
+      latch.await();
+      assertEquals(2, server.getCallQueueLen());
+
+      // unplug the callq
+      barrier.await();
+      barrierCall.get();
+
+      // verify correct ugi is used
+      String answer = remoteUserCall.get();
+      assertEquals(ugi.getUserName(), answer);
+
+      try {
+        exceptionCall.get();
+        fail("didn't throw");
+      } catch (IOException ioe) {
+        assertEquals(expectedIOE.getMessage(), ioe.getMessage());
+      }
+    } finally {
+      server.stop();
+    }
+  }
+
+  private <T> ExternalCall<T> newExtCall(UserGroupInformation ugi,
+      PrivilegedExceptionAction<T> callable) {
+    return new ExternalCall<T>(callable) {
+      @Override
+      public String getProtocol() {
+        return "test";
+      }
+      @Override
+      public UserGroupInformation getRemoteUser() {
+        return ugi;
+      }
+    };
+  }
+
   @Test
   public void testRpcMetrics() throws Exception {
     Server server;

+ 23 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java

@@ -102,4 +102,27 @@ public class ServerSocketUtil {
       }
     }
   }
+
+  /**
+   * Find the specified number of unique ports available.
+   * The ports are all closed afterwards,
+   * so other network services started may grab those same ports.
+   *
+   * @param numPorts number of required port nubmers
+   * @return array of available port numbers
+   * @throws IOException
+   */
+  public static int[] getPorts(int numPorts) throws IOException {
+    ServerSocket[] sockets = new ServerSocket[numPorts];
+    int[] ports = new int[numPorts];
+    for (int i = 0; i < numPorts; i++) {
+      ServerSocket sock = new ServerSocket(0);
+      sockets[i] = sock;
+      ports[i] = sock.getLocalPort();
+    }
+    for (ServerSocket sock : sockets) {
+      sock.close();
+    }
+    return ports;
+  }
 }

+ 4 - 29
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -50,7 +50,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.LinkedHashSet;
-import java.util.List;
 import java.util.Set;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
@@ -891,40 +890,16 @@ public class TestUserGroupInformation {
     ugi.addToken(new Text("regular-token"), token);
 
     // Now add cloned private token
-    ugi.addToken(new Text("private-token"), new Token.PrivateToken<TestTokenIdentifier>(token));
-    ugi.addToken(new Text("private-token1"), new Token.PrivateToken<TestTokenIdentifier>(token));
+    Text service = new Text("private-token");
+    ugi.addToken(service, token.privateClone(service));
+    Text service1 = new Text("private-token1");
+    ugi.addToken(service1, token.privateClone(service1));
 
     // Ensure only non-private tokens are returned
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();
     assertEquals(1, tokens.size());
   }
 
-  @Test(timeout = 30000)
-  public void testCopySubjectAndUgi() throws IOException {
-    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.SIMPLE, conf);
-    UserGroupInformation.setConfiguration(conf);
-    UserGroupInformation u1 = UserGroupInformation.getLoginUser();
-    assertNotNull(u1);
-    @SuppressWarnings("unchecked")
-    Token<? extends TokenIdentifier> tmpToken = mock(Token.class);
-    u1.addToken(tmpToken);
-
-    UserGroupInformation u2 = u1.copySubjectAndUgi();
-    assertEquals(u1.getAuthenticationMethod(), u2.getAuthenticationMethod());
-    assertNotSame(u1.getSubject(), u2.getSubject());
-    Credentials c1 = u1.getCredentials(), c2 = u2.getCredentials();
-    List<Text> sc1 = c1.getAllSecretKeys(), sc2 = c2.getAllSecretKeys();
-    assertArrayEquals(sc1.toArray(new Text[0]), sc2.toArray(new Text[0]));
-    Collection<Token<? extends TokenIdentifier>> ts1 = c1.getAllTokens(),
-        ts2 = c2.getAllTokens();
-    assertArrayEquals(ts1.toArray(new Token[0]), ts2.toArray(new Token[0]));
-    @SuppressWarnings("unchecked")
-    Token<? extends TokenIdentifier> token = mock(Token.class);
-    u2.addToken(token);
-    assertTrue(u2.getCredentials().getAllTokens().contains(token));
-    assertFalse(u1.getCredentials().getAllTokens().contains(token));
-  }
-
   /**
    * This test checks a race condition between getting and adding tokens for
    * the current user.  Calling UserGroupInformation.getCurrentUser() returns

+ 6 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java

@@ -141,10 +141,15 @@ public class TestSysInfoWindows {
   @Test(timeout = 10000)
   public void errorInGetSystemInfo() {
     SysInfoWindowsMock tester = new SysInfoWindowsMock();
-    // info str derived from windows shell command has \r\n termination
+    // info str derived from windows shell command is null
     tester.setSysinfoString(null);
     // call a method to refresh values
     tester.getAvailablePhysicalMemorySize();
+
+    // info str derived from windows shell command with no \r\n termination
+    tester.setSysinfoString("");
+    // call a method to refresh values
+    tester.getAvailablePhysicalMemorySize();
   }
 
 }

+ 66 - 10
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java

@@ -28,6 +28,8 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 import javax.ws.rs.Consumes;
@@ -68,6 +70,8 @@ public class KMS {
   private KeyProviderCryptoExtension provider;
   private KMSAudit kmsAudit;
 
+  private static final Logger LOG = LoggerFactory.getLogger(KMS.class);
+
   public KMS() throws Exception {
     provider = KMSWebApp.getKeyProvider();
     kmsAudit= KMSWebApp.getKMSAudit();
@@ -77,7 +81,7 @@ public class KMS {
       KMSOp operation) throws AccessControlException {
     KMSWebApp.getACLs().assertAccess(aclType, ugi, operation, null);
   }
-  
+
   private void assertAccess(KMSACLs.Type aclType, UserGroupInformation ugi,
       KMSOp operation, String key) throws AccessControlException {
     KMSWebApp.getACLs().assertAccess(aclType, ugi, operation, key);
@@ -100,6 +104,7 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   @SuppressWarnings("unchecked")
   public Response createKey(Map jsonKey) throws Exception {
+    LOG.trace("Entering createKey Method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
@@ -111,6 +116,9 @@ public class KMS {
                  ? (Integer) jsonKey.get(KMSRESTConstants.LENGTH_FIELD) : 0;
     String description = (String)
         jsonKey.get(KMSRESTConstants.DESCRIPTION_FIELD);
+    LOG.debug("Creating key with name {}, cipher being used{}, " +
+            "length of key {}, description of key {}", name, cipher,
+             length, description);
     Map<String, String> attributes = (Map<String, String>)
         jsonKey.get(KMSRESTConstants.ATTRIBUTES_FIELD);
     if (material != null) {
@@ -151,6 +159,7 @@ public class KMS {
     String requestURL = KMSMDCFilter.getURL();
     int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE);
     requestURL = requestURL.substring(0, idx);
+    LOG.trace("Exiting createKey Method.");
     return Response.created(getKeyURI(KMSRESTConstants.SERVICE_VERSION, name))
         .type(MediaType.APPLICATION_JSON)
         .header("Location", getKeyURI(requestURL, name)).entity(json).build();
@@ -160,11 +169,12 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   public Response deleteKey(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering deleteKey method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
     KMSClientProvider.checkNotEmpty(name, "name");
-
+    LOG.debug("Deleting key with name {}.", name);
     user.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
@@ -175,7 +185,7 @@ public class KMS {
     });
 
     kmsAudit.ok(user, KMSOp.DELETE_KEY, name, "");
-
+    LOG.trace("Exiting deleteKey method.");
     return Response.ok().build();
   }
 
@@ -185,10 +195,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response rolloverKey(@PathParam("name") final String name,
       Map jsonMaterial) throws Exception {
+    LOG.trace("Entering rolloverKey Method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.ROLL_NEW_VERSION, name);
     KMSClientProvider.checkNotEmpty(name, "name");
+    LOG.debug("Rolling key with name {}.", name);
     final String material = (String)
         jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
     if (material != null) {
@@ -216,6 +228,7 @@ public class KMS {
       keyVersion = removeKeyMaterial(keyVersion);
     }
     Map json = KMSServerJSONUtils.toJSON(keyVersion);
+    LOG.trace("Exiting rolloverKey Method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -224,6 +237,7 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeysMetadata(@QueryParam(KMSRESTConstants.KEY)
       List<String> keyNamesList) throws Exception {
+    LOG.trace("Entering getKeysMetadata method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     final String[] keyNames = keyNamesList.toArray(
@@ -241,6 +255,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(keyNames, keysMeta);
     kmsAudit.ok(user, KMSOp.GET_KEYS_METADATA, "");
+    LOG.trace("Exiting getKeysMetadata method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -248,6 +263,7 @@ public class KMS {
   @Path(KMSRESTConstants.KEYS_NAMES_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyNames() throws Exception {
+    LOG.trace("Entering getKeyNames method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.GET_KEYS, user, KMSOp.GET_KEYS);
@@ -262,6 +278,7 @@ public class KMS {
     );
 
     kmsAudit.ok(user, KMSOp.GET_KEYS, "");
+    LOG.trace("Exiting getKeyNames method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -269,6 +286,9 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   public Response getKey(@PathParam("name") String name)
       throws Exception {
+    LOG.trace("Entering getKey method.");
+    LOG.debug("Getting key information for key with name {}.", name);
+    LOG.trace("Exiting getKey method.");
     return getMetadata(name);
   }
 
@@ -278,10 +298,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getMetadata(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering getMetadata method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getAdminCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
+    LOG.debug("Getting metadata for key with name {}.", name);
 
     KeyProvider.Metadata metadata = user.doAs(
         new PrivilegedExceptionAction<KeyProvider.Metadata>() {
@@ -294,6 +316,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(name, metadata);
     kmsAudit.ok(user, KMSOp.GET_METADATA, name, "");
+    LOG.trace("Exiting getMetadata method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -303,10 +326,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getCurrentVersion(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering getCurrentVersion method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
+    LOG.debug("Getting key version for key with name {}.", name);
 
     KeyVersion keyVersion = user.doAs(
         new PrivilegedExceptionAction<KeyVersion>() {
@@ -319,6 +344,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(keyVersion);
     kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
+    LOG.trace("Exiting getCurrentVersion method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -327,10 +353,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyVersion(
       @PathParam("versionName") final String versionName) throws Exception {
+    LOG.trace("Entering getKeyVersion method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
+    LOG.debug("Getting key with version name {}.", versionName);
 
     KeyVersion keyVersion = user.doAs(
         new PrivilegedExceptionAction<KeyVersion>() {
@@ -345,6 +373,7 @@ public class KMS {
       kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
     }
     Object json = KMSServerJSONUtils.toJSON(keyVersion);
+    LOG.trace("Exiting getKeyVersion method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -359,12 +388,16 @@ public class KMS {
           @DefaultValue("1")
           @QueryParam(KMSRESTConstants.EEK_NUM_KEYS) final int numKeys)
           throws Exception {
+    LOG.trace("Entering generateEncryptedKeys method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSClientProvider.checkNotNull(edekOp, "eekOp");
+    LOG.debug("Generating encrypted key with name {}," +
+            " the edek Operation is {}.", name, edekOp);
 
     Object retJSON;
     if (edekOp.equals(KMSRESTConstants.EEK_GENERATE)) {
+      LOG.debug("edek Operation is Generate.");
       assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.GENERATE_EEK, name);
 
       final List<EncryptedKeyVersion> retEdeks =
@@ -375,6 +408,8 @@ public class KMS {
             new PrivilegedExceptionAction<Void>() {
               @Override
               public Void run() throws Exception {
+                  LOG.debug("Generated Encrypted key for {} number of keys.",
+                          numKeys);
                 for (int i = 0; i < numKeys; i++) {
                   retEdeks.add(provider.generateEncryptedKey(name));
                 }
@@ -384,6 +419,7 @@ public class KMS {
         );
 
       } catch (Exception e) {
+        LOG.error("Exception in generateEncryptedKeys:", e);
         throw new IOException(e);
       }
       kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
@@ -392,11 +428,18 @@ public class KMS {
         ((ArrayList)retJSON).add(KMSServerJSONUtils.toJSON(edek));
       }
     } else {
-      throw new IllegalArgumentException("Wrong " + KMSRESTConstants.EEK_OP +
-          " value, it must be " + KMSRESTConstants.EEK_GENERATE + " or " +
-          KMSRESTConstants.EEK_DECRYPT);
+      StringBuilder error;
+      error = new StringBuilder("IllegalArgumentException Wrong ");
+      error.append(KMSRESTConstants.EEK_OP);
+      error.append(" value, it must be ");
+      error.append(KMSRESTConstants.EEK_GENERATE);
+      error.append(" or ");
+      error.append(KMSRESTConstants.EEK_DECRYPT);
+      LOG.error(error.toString());
+      throw new IllegalArgumentException(error.toString());
     }
     KMSWebApp.getGenerateEEKCallsMeter().mark();
+    LOG.trace("Exiting generateEncryptedKeys method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
         .build();
   }
@@ -411,14 +454,17 @@ public class KMS {
       @QueryParam(KMSRESTConstants.EEK_OP) String eekOp,
       Map jsonPayload)
       throws Exception {
+    LOG.trace("Entering decryptEncryptedKey method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSClientProvider.checkNotNull(eekOp, "eekOp");
+    LOG.debug("Decrypting key for {}, the edek Operation is {}.",
+            versionName, eekOp);
 
     final String keyName = (String) jsonPayload.get(
         KMSRESTConstants.NAME_FIELD);
     String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
-    String encMaterialStr = 
+    String encMaterialStr =
         (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
     Object retJSON;
     if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
@@ -445,11 +491,18 @@ public class KMS {
       retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
       kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
     } else {
-      throw new IllegalArgumentException("Wrong " + KMSRESTConstants.EEK_OP +
-          " value, it must be " + KMSRESTConstants.EEK_GENERATE + " or " +
-          KMSRESTConstants.EEK_DECRYPT);
+      StringBuilder error;
+      error = new StringBuilder("IllegalArgumentException Wrong ");
+      error.append(KMSRESTConstants.EEK_OP);
+      error.append(" value, it must be ");
+      error.append(KMSRESTConstants.EEK_GENERATE);
+      error.append(" or ");
+      error.append(KMSRESTConstants.EEK_DECRYPT);
+      LOG.error(error.toString());
+      throw new IllegalArgumentException(error.toString());
     }
     KMSWebApp.getDecryptEEKCallsMeter().mark();
+    LOG.trace("Exiting decryptEncryptedKey method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
         .build();
   }
@@ -460,10 +513,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyVersions(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering getKeyVersions method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
+    LOG.debug("Getting key versions for key {}", name);
 
     List<KeyVersion> ret = user.doAs(
         new PrivilegedExceptionAction<List<KeyVersion>>() {
@@ -476,6 +531,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(ret);
     kmsAudit.ok(user, KMSOp.GET_KEY_VERSIONS, name, "");
+    LOG.trace("Exiting getKeyVersions method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 

+ 2 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.PropertyConfigurator;
@@ -121,6 +122,7 @@ public class KMSWebApp implements ServletContextListener {
       }
       kmsConf = KMSConfiguration.getKMSConf();
       initLogging(confDir);
+      UserGroupInformation.setConfiguration(kmsConf);
       LOG.info("-------------------------------------------------------------");
       LOG.info("  Java runtime version : {}", System.getProperty(
           "java.runtime.version"));

+ 40 - 36
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java

@@ -143,11 +143,31 @@ public class TestKMS {
   }
 
   protected Configuration createBaseKMSConf(File keyStoreDir) throws Exception {
-    Configuration conf = new Configuration(false);
-    conf.set(KMSConfiguration.KEY_PROVIDER_URI,
+    return createBaseKMSConf(keyStoreDir, null);
+  }
+
+  /**
+   * The Configuration object is shared by both KMS client and server in unit
+   * tests because UGI gets/sets it to a static variable.
+   * As a workaround, make sure the client configurations are copied to server
+   * so that client can read them.
+   * @param keyStoreDir where keystore is located.
+   * @param conf KMS client configuration
+   * @return KMS server configuration based on client.
+   * @throws Exception
+   */
+  protected Configuration createBaseKMSConf(File keyStoreDir,
+      Configuration conf) throws Exception {
+    Configuration newConf;
+    if (conf == null) {
+      newConf = new Configuration(false);
+    } else {
+      newConf = new Configuration(conf);
+    }
+    newConf.set(KMSConfiguration.KEY_PROVIDER_URI,
         "jceks://file@" + new Path(keyStoreDir.getAbsolutePath(), "kms.keystore").toUri());
-    conf.set("hadoop.kms.authentication.type", "simple");
-    return conf;
+    newConf.set("hadoop.kms.authentication.type", "simple");
+    return newConf;
   }
 
   public static void writeConf(File confDir, Configuration conf)
@@ -280,9 +300,8 @@ public class TestKMS {
     if (kerberos) {
       conf.set("hadoop.security.authentication", "kerberos");
     }
-    UserGroupInformation.setConfiguration(conf);
     File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
 
     final String keystore;
     final String password;
@@ -404,9 +423,8 @@ public class TestKMS {
     final String specialKey = "key %^[\n{]}|\"<>\\";
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     File confDir = getTestDir();
-    conf = createBaseKMSConf(confDir);
+    conf = createBaseKMSConf(confDir, conf);
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + specialKey + ".ALL", "*");
     writeConf(confDir, conf);
 
@@ -439,9 +457,8 @@ public class TestKMS {
   public void testKMSProvider() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     File confDir = getTestDir();
-    conf = createBaseKMSConf(confDir);
+    conf = createBaseKMSConf(confDir, conf);
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "k1.ALL", "*");
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "k2.MANAGEMENT", "*");
     conf.set(KeyAuthorizationKeyProvider.KEY_ACL + "k2.READ", "*");
@@ -699,9 +716,8 @@ public class TestKMS {
   public void testKeyACLs() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -977,9 +993,8 @@ public class TestKMS {
   public void doKMSRestart(boolean useKrb) throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     if (useKrb) {
       conf.set("hadoop.kms.authentication.type", "kerberos");
     }
@@ -1057,9 +1072,8 @@ public class TestKMS {
   public void testKMSAuthFailureRetry() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
     conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
@@ -1151,9 +1165,8 @@ public class TestKMS {
   public void testACLs() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -1461,9 +1474,8 @@ public class TestKMS {
   public void testKMSBlackList() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -1550,9 +1562,8 @@ public class TestKMS {
   public void testServicePrincipalACLs() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -1676,9 +1687,8 @@ public class TestKMS {
   public void testDelegationTokenAccess() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -1759,9 +1769,8 @@ public class TestKMS {
 
   private void testDelegationTokensOps(Configuration conf,
       final boolean useKrb) throws Exception {
-    UserGroupInformation.setConfiguration(conf);
     File confDir = getTestDir();
-    conf = createBaseKMSConf(confDir);
+    conf = createBaseKMSConf(confDir, conf);
     if (useKrb) {
       conf.set("hadoop.kms.authentication.type", "kerberos");
       conf.set("hadoop.kms.authentication.kerberos.keytab",
@@ -1885,9 +1894,8 @@ public class TestKMS {
   @Test
   public void testDelegationTokensUpdatedInUGI() throws Exception {
     Configuration conf = new Configuration();
-    UserGroupInformation.setConfiguration(conf);
     File confDir = getTestDir();
-    conf = createBaseKMSConf(confDir);
+    conf = createBaseKMSConf(confDir, conf);
     conf.set(
         "hadoop.kms.authentication.delegation-token.max-lifetime.sec", "5");
     conf.set(
@@ -2024,9 +2032,8 @@ public class TestKMS {
 
       Configuration conf = new Configuration();
       conf.set("hadoop.security.authentication", "kerberos");
-      UserGroupInformation.setConfiguration(conf);
       final File testDir = getTestDir();
-      conf = createBaseKMSConf(testDir);
+      conf = createBaseKMSConf(testDir, conf);
       conf.set("hadoop.kms.authentication.type", "kerberos");
       conf.set("hadoop.kms.authentication.kerberos.keytab", keytab.getAbsolutePath());
       conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
@@ -2114,9 +2121,8 @@ public class TestKMS {
   public void doProxyUserTest(final boolean kerberos) throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     if (kerberos) {
       conf.set("hadoop.kms.authentication.type", "kerberos");
     }
@@ -2226,9 +2232,8 @@ public class TestKMS {
 
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -2286,9 +2291,8 @@ public class TestKMS {
   public void doWebHDFSProxyUserTest(final boolean kerberos) throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir, conf);
     if (kerberos) {
       conf.set("hadoop.kms.authentication.type", "kerberos");
     }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/pom.xml

@@ -107,6 +107,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+      <dependency>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+      </dependency>
   </dependencies>
 
   <build>

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1710,6 +1710,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  @VisibleForTesting
+  public DataEncryptionKey getEncryptionKey() {
+    return encryptionKey;
+  }
+
   /**
    * Get the checksum of the whole file or a range of the file. Note that the
    * range always starts from the beginning of the file. The file can be
@@ -2594,8 +2599,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try (TraceScope ignored = newPathTraceScope("getEZForPath", src)) {
       return namenode.getEZForPath(src);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(FileNotFoundException.class,
-          AccessControlException.class, UnresolvedPathException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          UnresolvedPathException.class);
     }
   }
 

+ 109 - 37
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

@@ -116,6 +116,89 @@ import javax.annotation.Nonnull;
 class DataStreamer extends Daemon {
   static final Logger LOG = LoggerFactory.getLogger(DataStreamer.class);
 
+  private class RefetchEncryptionKeyPolicy {
+    private int fetchEncryptionKeyTimes = 0;
+    private InvalidEncryptionKeyException lastException;
+    private final DatanodeInfo src;
+
+    RefetchEncryptionKeyPolicy(DatanodeInfo src) {
+      this.src = src;
+    }
+    boolean continueRetryingOrThrow() throws InvalidEncryptionKeyException {
+      if (fetchEncryptionKeyTimes >= 2) {
+        // hit the same exception twice connecting to the node, so
+        // throw the exception and exclude the node.
+        throw lastException;
+      }
+      // Don't exclude this node just yet.
+      // Try again with a new encryption key.
+      LOG.info("Will fetch a new encryption key and retry, "
+          + "encryption key was invalid when connecting to "
+          + this.src + ": ", lastException);
+      // The encryption key used is invalid.
+      dfsClient.clearDataEncryptionKey();
+      return true;
+    }
+
+    /**
+     * Record a connection exception.
+     * @param e
+     * @throws InvalidEncryptionKeyException
+     */
+    void recordFailure(final InvalidEncryptionKeyException e)
+        throws InvalidEncryptionKeyException {
+      fetchEncryptionKeyTimes++;
+      lastException = e;
+    }
+  }
+
+  private class StreamerStreams implements java.io.Closeable {
+    private Socket sock = null;
+    private DataOutputStream out = null;
+    private DataInputStream in = null;
+
+    StreamerStreams(final DatanodeInfo src,
+        final long writeTimeout, final long readTimeout,
+        final Token<BlockTokenIdentifier> blockToken)
+        throws IOException {
+      sock = createSocketForPipeline(src, 2, dfsClient);
+
+      OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
+      InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
+      IOStreamPair saslStreams = dfsClient.saslClient
+          .socketSend(sock, unbufOut, unbufIn, dfsClient, blockToken, src);
+      unbufOut = saslStreams.out;
+      unbufIn = saslStreams.in;
+      out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+          DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+      in = new DataInputStream(unbufIn);
+    }
+
+    void sendTransferBlock(final DatanodeInfo[] targets,
+        final StorageType[] targetStorageTypes,
+        final Token<BlockTokenIdentifier> blockToken) throws IOException {
+      //send the TRANSFER_BLOCK request
+      new Sender(out)
+          .transferBlock(block, blockToken, dfsClient.clientName, targets,
+              targetStorageTypes);
+      out.flush();
+      //ack
+      BlockOpResponseProto transferResponse = BlockOpResponseProto
+          .parseFrom(PBHelperClient.vintPrefixed(in));
+      if (SUCCESS != transferResponse.getStatus()) {
+        throw new IOException("Failed to add a datanode. Response status: "
+            + transferResponse.getStatus());
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      IOUtils.closeStream(in);
+      IOUtils.closeStream(out);
+      IOUtils.closeSocket(sock);
+    }
+  }
+
   /**
    * Create a socket for a write pipeline
    *
@@ -1270,50 +1353,39 @@ class DataStreamer extends Daemon {
         new IOException("Failed to add a node");
   }
 
+  private long computeTransferWriteTimeout() {
+    return dfsClient.getDatanodeWriteTimeout(2);
+  }
+  private long computeTransferReadTimeout() {
+    // transfer timeout multiplier based on the transfer size
+    // One per 200 packets = 12.8MB. Minimum is 2.
+    int multi = 2
+        + (int) (bytesSent / dfsClient.getConf().getWritePacketSize()) / 200;
+    return dfsClient.getDatanodeReadTimeout(multi);
+  }
+
   private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
                         final StorageType[] targetStorageTypes,
                         final Token<BlockTokenIdentifier> blockToken)
       throws IOException {
     //transfer replica to the new datanode
-    Socket sock = null;
-    DataOutputStream out = null;
-    DataInputStream in = null;
-    try {
-      sock = createSocketForPipeline(src, 2, dfsClient);
-      final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
-
-      // transfer timeout multiplier based on the transfer size
-      // One per 200 packets = 12.8MB. Minimum is 2.
-      int multi = 2 + (int)(bytesSent /dfsClient.getConf().getWritePacketSize())
-          / 200;
-      final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
-
-      OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
-      InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
-      IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
-          unbufOut, unbufIn, dfsClient, blockToken, src);
-      unbufOut = saslStreams.out;
-      unbufIn = saslStreams.in;
-      out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-          DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
-      in = new DataInputStream(unbufIn);
-
-      //send the TRANSFER_BLOCK request
-      new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
-          targets, targetStorageTypes);
-      out.flush();
+    RefetchEncryptionKeyPolicy policy = new RefetchEncryptionKeyPolicy(src);
+    do {
+      StreamerStreams streams = null;
+      try {
+        final long writeTimeout = computeTransferWriteTimeout();
+        final long readTimeout = computeTransferReadTimeout();
 
-      //ack
-      BlockOpResponseProto response =
-          BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
-      if (SUCCESS != response.getStatus()) {
-        throw new IOException("Failed to add a datanode");
+        streams = new StreamerStreams(src, writeTimeout, readTimeout,
+            blockToken);
+        streams.sendTransferBlock(targets, targetStorageTypes, blockToken);
+        return;
+      } catch (InvalidEncryptionKeyException e) {
+        policy.recordFailure(e);
+      } finally {
+        IOUtils.closeStream(streams);
       }
-    } finally {
-      IOUtils.closeStream(in);
-      IOUtils.closeStream(out);
-      IOUtils.closeSocket(sock);
-    }
+    } while (policy.continueRetryingOrThrow());
   }
 
   /**

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSLinkResolver;
 import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemLinkResolver;
@@ -2204,6 +2205,35 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.listEncryptionZones();
   }
 
+  /* HDFS only */
+  public FileEncryptionInfo getFileEncryptionInfo(final Path path)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<FileEncryptionInfo>() {
+      @Override
+      public FileEncryptionInfo doCall(final Path p) throws IOException {
+        final HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
+        if (fi == null) {
+          throw new FileNotFoundException("File does not exist: " + p);
+        }
+        return fi.getFileEncryptionInfo();
+      }
+
+      @Override
+      public FileEncryptionInfo next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          return myDfs.getFileEncryptionInfo(p);
+        }
+        throw new UnsupportedOperationException(
+            "Cannot call getFileEncryptionInfo"
+                + " on a symlink to a non-DistributedFileSystem: " + path
+                + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+
   @Override
   public void setXAttr(Path path, final String name, final byte[] value,
       final EnumSet<XAttrSetFlag> flag) throws IOException {

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/CreateEncryptionZoneFlag.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/CreateEncryptionZoneFlag.java

@@ -68,4 +68,3 @@ public enum CreateEncryptionZoneFlag {
     return mode;
   }
 }
-

+ 41 - 29
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,29 +46,28 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 /**
  * The public API for performing administrative functions on HDFS. Those writing
  * applications against HDFS should prefer this interface to directly accessing
  * functionality in DistributedFileSystem or DFSClient.
- * 
- * Note that this is distinct from the similarly-named {@link DFSAdmin}, which
+ *
+ * Note that this is distinct from the similarly-named DFSAdmin, which
  * is a class that provides the functionality for the CLI `hdfs dfsadmin ...'
  * commands.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class HdfsAdmin {
-  
+
   private DistributedFileSystem dfs;
   private static final FsPermission TRASH_PERMISSION = new FsPermission(
       FsAction.ALL, FsAction.ALL, FsAction.ALL, true);
-  
+
   /**
    * Create a new HdfsAdmin client.
-   * 
+   *
    * @param uri the unique URI of the HDFS file system to administer
    * @param conf configuration
    * @throws IOException in the event the file system could not be created
@@ -80,11 +80,11 @@ public class HdfsAdmin {
       dfs = (DistributedFileSystem)fs;
     }
   }
-  
+
   /**
    * Set the namespace quota (count of files, directories, and sym links) for a
    * directory.
-   * 
+   *
    * @param src the path to set the quota for
    * @param quota the value to set for the quota
    * @throws IOException in the event of error
@@ -92,22 +92,22 @@ public class HdfsAdmin {
   public void setQuota(Path src, long quota) throws IOException {
     dfs.setQuota(src, quota, HdfsConstants.QUOTA_DONT_SET);
   }
-  
+
   /**
    * Clear the namespace quota (count of files, directories and sym links) for a
    * directory.
-   * 
+   *
    * @param src the path to clear the quota of
    * @throws IOException in the event of error
    */
   public void clearQuota(Path src) throws IOException {
     dfs.setQuota(src, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET);
   }
-  
+
   /**
    * Set the storage space quota (size of files) for a directory. Note that
    * directories and sym links do not occupy storage space.
-   * 
+   *
    * @param src the path to set the space quota of
    * @param spaceQuota the value to set for the space quota
    * @throws IOException in the event of error
@@ -115,11 +115,11 @@ public class HdfsAdmin {
   public void setSpaceQuota(Path src, long spaceQuota) throws IOException {
     dfs.setQuota(src, HdfsConstants.QUOTA_DONT_SET, spaceQuota);
   }
-  
+
   /**
    * Clear the storage space quota (size of files) for a directory. Note that
    * directories and sym links do not occupy storage space.
-   * 
+   *
    * @param src the path to clear the space quota of
    * @throws IOException in the event of error
    */
@@ -152,7 +152,7 @@ public class HdfsAdmin {
   public void clearQuotaByStorageType(Path src, StorageType type) throws IOException {
     dfs.setQuotaByStorageType(src, type, HdfsConstants.QUOTA_RESET);
   }
-  
+
   /**
    * Allow snapshot on a directory.
    * @param path The path of the directory where snapshots will be taken.
@@ -160,7 +160,7 @@ public class HdfsAdmin {
   public void allowSnapshot(Path path) throws IOException {
     dfs.allowSnapshot(path);
   }
-  
+
   /**
    * Disallow snapshot on a directory.
    * @param path The path of the snapshottable directory.
@@ -171,7 +171,7 @@ public class HdfsAdmin {
 
   /**
    * Add a new CacheDirectiveInfo.
-   * 
+   *
    * @param info Information about a directive to add.
    * @param flags {@link CacheFlag}s to use for this operation.
    * @return the ID of the directive that was created.
@@ -181,10 +181,10 @@ public class HdfsAdmin {
       EnumSet<CacheFlag> flags) throws IOException {
   return dfs.addCacheDirective(info, flags);
   }
-  
+
   /**
    * Modify a CacheDirective.
-   * 
+   *
    * @param info Information about the directive to modify. You must set the ID
    *          to indicate which CacheDirective you want to modify.
    * @param flags {@link CacheFlag}s to use for this operation.
@@ -197,7 +197,7 @@ public class HdfsAdmin {
 
   /**
    * Remove a CacheDirective.
-   * 
+   *
    * @param id identifier of the CacheDirectiveInfo to remove
    * @throws IOException if the directive could not be removed
    */
@@ -208,7 +208,7 @@ public class HdfsAdmin {
 
   /**
    * List cache directives. Incrementally fetches results from the server.
-   * 
+   *
    * @param filter Filter parameters to use when listing the directives, null to
    *               list all directives visible to us.
    * @return A RemoteIterator which returns CacheDirectiveInfo objects.
@@ -223,7 +223,7 @@ public class HdfsAdmin {
    *
    * @param info
    *          The request to add a cache pool.
-   * @throws IOException 
+   * @throws IOException
    *          If the request could not be completed.
    */
   public void addCachePool(CachePoolInfo info) throws IOException {
@@ -235,19 +235,19 @@ public class HdfsAdmin {
    *
    * @param info
    *          The request to modify a cache pool.
-   * @throws IOException 
+   * @throws IOException
    *          If the request could not be completed.
    */
   public void modifyCachePool(CachePoolInfo info) throws IOException {
     dfs.modifyCachePool(info);
   }
-    
+
   /**
    * Remove a cache pool.
    *
    * @param poolName
    *          Name of the cache pool to remove.
-   * @throws IOException 
+   * @throws IOException
    *          if the cache pool did not exist, or could not be removed.
    */
   public void removeCachePool(String poolName) throws IOException {
@@ -329,14 +329,13 @@ public class HdfsAdmin {
    * Get the path of the encryption zone for a given file or directory.
    *
    * @param path The path to get the ez for.
-   *
-   * @return The EncryptionZone of the ez, or null if path is not in an ez.
+   * @return An EncryptionZone, or null if path does not exist or is not in an
+   * ez.
    * @throws IOException            if there was a general IO exception
    * @throws AccessControlException if the caller does not have access to path
-   * @throws FileNotFoundException  if the path does not exist
    */
   public EncryptionZone getEncryptionZoneForPath(Path path)
-    throws IOException, AccessControlException, FileNotFoundException {
+      throws IOException, AccessControlException {
     return dfs.getEZForPath(path);
   }
 
@@ -355,6 +354,19 @@ public class HdfsAdmin {
     return dfs.listEncryptionZones();
   }
 
+  /**
+   * Returns the FileEncryptionInfo on the HdfsFileStatus for the given path.
+   * The return value can be null if the path points to a directory, or a file
+   * that is not in an encryption zone.
+   *
+   * @throws FileNotFoundException if the path does not exist.
+   * @throws AccessControlException if no execute permission on parent path.
+   */
+  public FileEncryptionInfo getFileEncryptionInfo(final Path path)
+      throws IOException {
+    return dfs.getFileEncryptionInfo(path);
+  }
+
   /**
    * Exposes a stream of namesystem events. Only events occurring after the
    * stream is created are available.

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsUtils.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsUtils.java

@@ -20,8 +20,7 @@ package org.apache.hadoop.hdfs.client;
 import java.io.IOException;
 import java.net.URI;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -30,7 +29,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The public utility API for HDFS.
@@ -38,7 +38,7 @@ import org.apache.hadoop.io.IOUtils;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class HdfsUtils {
-  private static final Log LOG = LogFactory.getLog(HdfsUtils.class);
+  public static final Logger LOG = LoggerFactory.getLogger(HdfsUtils.class);
 
   /**
    * Is the HDFS healthy?
@@ -54,7 +54,7 @@ public class HdfsUtils {
       throw new IllegalArgumentException("The scheme is not "
           + HdfsConstants.HDFS_URI_SCHEME + ", uri=" + uri);
     }
-    
+
     final Configuration conf = new Configuration();
     //disable FileSystem cache
     conf.setBoolean(String.format("fs.%s.impl.disable.cache", scheme), true);
@@ -80,7 +80,7 @@ public class HdfsUtils {
       }
       return false;
     } finally {
-      IOUtils.cleanup(LOG, fs);
+      IOUtils.closeQuietly(fs);
     }
   }
 }

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/package-info.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/package-info.java


+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java

@@ -19,10 +19,10 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.htrace.fasterxml.jackson.annotation.JsonInclude;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectReader;
 

+ 55 - 33
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java

@@ -26,13 +26,14 @@ import java.nio.MappedByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.TreeMap;
+import java.util.NoSuchElementException;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
@@ -107,16 +108,20 @@ public class ShortCircuitCache implements Closeable {
 
         int numDemoted = demoteOldEvictableMmaped(curMs);
         int numPurged = 0;
-        Long evictionTimeNs = (long) 0;
+        Long evictionTimeNs;
         while (true) {
-          Entry<Long, ShortCircuitReplica> entry =
-              evictable.ceilingEntry(evictionTimeNs);
-          if (entry == null) break;
-          evictionTimeNs = entry.getKey();
+          Object eldestKey;
+          try {
+            eldestKey = evictable.firstKey();
+          } catch (NoSuchElementException e) {
+            break;
+          }
+          evictionTimeNs = (Long)eldestKey;
           long evictionTimeMs =
               TimeUnit.MILLISECONDS.convert(evictionTimeNs, TimeUnit.NANOSECONDS);
           if (evictionTimeMs + maxNonMmappedEvictableLifespanMs >= curMs) break;
-          ShortCircuitReplica replica = entry.getValue();
+          ShortCircuitReplica replica = (ShortCircuitReplica)evictable.get(
+              eldestKey);
           if (LOG.isTraceEnabled()) {
             LOG.trace("CacheCleaner: purging " + replica + ": " +
                 StringUtils.getStackTrace(Thread.currentThread()));
@@ -263,11 +268,11 @@ public class ShortCircuitCache implements Closeable {
   private CacheCleaner cacheCleaner;
 
   /**
-   * Tree of evictable elements.
+   * LinkedMap of evictable elements.
    *
    * Maps (unique) insertion time in nanoseconds to the element.
    */
-  private final TreeMap<Long, ShortCircuitReplica> evictable = new TreeMap<>();
+  private final LinkedMap evictable = new LinkedMap();
 
   /**
    * Maximum total size of the cache, including both mmapped and
@@ -281,12 +286,11 @@ public class ShortCircuitCache implements Closeable {
   private long maxNonMmappedEvictableLifespanMs;
 
   /**
-   * Tree of mmaped evictable elements.
+   * LinkedMap of mmaped evictable elements.
    *
    * Maps (unique) insertion time in nanoseconds to the element.
    */
-  private final TreeMap<Long, ShortCircuitReplica> evictableMmapped =
-      new TreeMap<>();
+  private final LinkedMap evictableMmapped = new LinkedMap();
 
   /**
    * Maximum number of mmaped evictable elements.
@@ -482,13 +486,16 @@ public class ShortCircuitCache implements Closeable {
   private int demoteOldEvictableMmaped(long now) {
     int numDemoted = 0;
     boolean needMoreSpace = false;
-    Long evictionTimeNs = (long) 0;
+    Long evictionTimeNs;
 
     while (true) {
-      Entry<Long, ShortCircuitReplica> entry =
-          evictableMmapped.ceilingEntry(evictionTimeNs);
-      if (entry == null) break;
-      evictionTimeNs = entry.getKey();
+      Object eldestKey;
+      try {
+        eldestKey = evictableMmapped.firstKey();
+      } catch (NoSuchElementException e) {
+        break;
+      }
+      evictionTimeNs = (Long)eldestKey;
       long evictionTimeMs =
           TimeUnit.MILLISECONDS.convert(evictionTimeNs, TimeUnit.NANOSECONDS);
       if (evictionTimeMs + maxEvictableMmapedLifespanMs >= now) {
@@ -497,7 +504,8 @@ public class ShortCircuitCache implements Closeable {
         }
         needMoreSpace = true;
       }
-      ShortCircuitReplica replica = entry.getValue();
+      ShortCircuitReplica replica = (ShortCircuitReplica)evictableMmapped.get(
+          eldestKey);
       if (LOG.isTraceEnabled()) {
         String rationale = needMoreSpace ? "because we need more space" :
             "because it's too old";
@@ -527,10 +535,15 @@ public class ShortCircuitCache implements Closeable {
         return;
       }
       ShortCircuitReplica replica;
-      if (evictableSize == 0) {
-        replica = evictableMmapped.firstEntry().getValue();
-      } else {
-        replica = evictable.firstEntry().getValue();
+      try {
+        if (evictableSize == 0) {
+          replica = (ShortCircuitReplica)evictableMmapped.get(evictableMmapped
+              .firstKey());
+        } else {
+          replica = (ShortCircuitReplica)evictable.get(evictable.firstKey());
+        }
+      } catch (NoSuchElementException e) {
+        break;
       }
       if (LOG.isTraceEnabled()) {
         LOG.trace(this + ": trimEvictionMaps is purging " + replica +
@@ -573,10 +586,11 @@ public class ShortCircuitCache implements Closeable {
    * @param map       The map to remove it from.
    */
   private void removeEvictable(ShortCircuitReplica replica,
-      TreeMap<Long, ShortCircuitReplica> map) {
+      LinkedMap map) {
     Long evictableTimeNs = replica.getEvictableTimeNs();
     Preconditions.checkNotNull(evictableTimeNs);
-    ShortCircuitReplica removed = map.remove(evictableTimeNs);
+    ShortCircuitReplica removed = (ShortCircuitReplica)map.remove(
+        evictableTimeNs);
     Preconditions.checkState(removed == replica,
         "failed to make %s unevictable", replica);
     replica.setEvictableTimeNs(null);
@@ -593,7 +607,7 @@ public class ShortCircuitCache implements Closeable {
    * @param map              The map to insert it into.
    */
   private void insertEvictable(Long evictionTimeNs,
-      ShortCircuitReplica replica, TreeMap<Long, ShortCircuitReplica> map) {
+      ShortCircuitReplica replica, LinkedMap map) {
     while (map.containsKey(evictionTimeNs)) {
       evictionTimeNs++;
     }
@@ -861,14 +875,22 @@ public class ShortCircuitCache implements Closeable {
       IOUtilsClient.cleanup(LOG, cacheCleaner);
       // Purge all replicas.
       while (true) {
-        Entry<Long, ShortCircuitReplica> entry = evictable.firstEntry();
-        if (entry == null) break;
-        purge(entry.getValue());
+        Object eldestKey;
+        try {
+          eldestKey = evictable.firstKey();
+        } catch (NoSuchElementException e) {
+          break;
+        }
+        purge((ShortCircuitReplica)evictable.get(eldestKey));
       }
       while (true) {
-        Entry<Long, ShortCircuitReplica> entry = evictableMmapped.firstEntry();
-        if (entry == null) break;
-        purge(entry.getValue());
+        Object eldestKey;
+        try {
+          eldestKey = evictableMmapped.firstKey();
+        } catch (NoSuchElementException e) {
+          break;
+        }
+        purge((ShortCircuitReplica)evictableMmapped.get(eldestKey));
       }
     } finally {
       lock.unlock();
@@ -909,8 +931,8 @@ public class ShortCircuitCache implements Closeable {
     void visit(int numOutstandingMmaps,
         Map<ExtendedBlockId, ShortCircuitReplica> replicas,
         Map<ExtendedBlockId, InvalidToken> failedLoads,
-        Map<Long, ShortCircuitReplica> evictable,
-        Map<Long, ShortCircuitReplica> evictableMmapped);
+        LinkedMap evictable,
+        LinkedMap evictableMmapped);
   }
 
   @VisibleForTesting // ONLY for testing

+ 20 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java

@@ -20,11 +20,11 @@ package org.apache.hadoop.hdfs.web.resources;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys
     .DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
 
+import java.util.Iterator;
 import java.util.List;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.commons.lang.StringUtils;
 
 /** AclPermission parameter. */
 public class AclPermissionParam extends StringParam {
@@ -63,7 +63,24 @@ public class AclPermissionParam extends StringParam {
   /**
    * @return parse {@code aclEntry} and return aclspec
    */
-  private static String parseAclSpec(List<AclEntry> aclEntry) {
-    return StringUtils.join(aclEntry, ",");
+  private static String parseAclSpec(List<AclEntry> aclEntries) {
+    if (aclEntries == null) {
+      return null;
+    }
+    if (aclEntries.isEmpty()) {
+      return "";
+    }
+    if (aclEntries.size() == 1) {
+      AclEntry entry = aclEntries.get(0);
+      return entry == null ? "" : entry.toStringStable();
+    }
+    StringBuilder sb = new StringBuilder();
+    Iterator<AclEntry> iter = aclEntries.iterator();
+    sb.append(iter.next().toStringStable());
+    while (iter.hasNext()) {
+      AclEntry entry = iter.next();
+      sb.append(',').append(entry == null ? "" : entry.toStringStable());
+    }
+    return sb.toString();
   }
 }

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml

@@ -299,7 +299,6 @@
             </goals>
             <phase>site</phase>
             <configuration>
-              <linksource>true</linksource>
               <quiet>true</quiet>
               <verbose>false</verbose>
               <source>${maven.compile.source}</source>

File diff suppressed because it is too large
+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.7.2.xml


+ 0 - 66
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/README.txt

@@ -1,66 +0,0 @@
-This module provides a BookKeeper backend for HFDS Namenode write
-ahead logging.  
-
-BookKeeper is a highly available distributed write ahead logging
-system. For more details, see
-   
-    http://zookeeper.apache.org/bookkeeper
-
--------------------------------------------------------------------------------
-How do I build?
-
- To generate the distribution packages for BK journal, do the
- following.
-
-   $ mvn clean package -Pdist
-
- This will generate a jar with all the dependencies needed by the journal
- manager, 
-
- target/hadoop-hdfs-bkjournal-<VERSION>.jar
-
- Note that the -Pdist part of the build command is important, as otherwise
- the dependencies would not be packaged in the jar. 
-
--------------------------------------------------------------------------------
-How do I use the BookKeeper Journal?
-
- To run a HDFS namenode using BookKeeper as a backend, copy the bkjournal
- jar, generated above, into the lib directory of hdfs. In the standard 
- distribution of HDFS, this is at $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
-
-  cp target/hadoop-hdfs-bkjournal-<VERSION>.jar \
-    $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
-
- Then, in hdfs-site.xml, set the following properties.
-
-   <property>
-     <name>dfs.namenode.edits.dir</name>
-     <value>bookkeeper://localhost:2181/bkjournal,file:///path/for/edits</value>
-   </property>
-
-   <property>
-     <name>dfs.namenode.edits.journal-plugin.bookkeeper</name>
-     <value>org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager</value>
-   </property>
-
- In this example, the namenode is configured to use 2 write ahead
- logging devices. One writes to BookKeeper and the other to a local
- file system. At the moment is is not possible to only write to 
- BookKeeper, as the resource checker explicitly checked for local
- disks currently.
-
- The given example, configures the namenode to look for the journal
- metadata at the path /bkjournal on the a standalone zookeeper ensemble
- at localhost:2181. To configure a multiple host zookeeper ensemble,
- separate the hosts with semicolons. For example, if you have 3
- zookeeper servers, zk1, zk2 & zk3, each listening on port 2181, you
- would specify this with 
-  
-   bookkeeper://zk1:2181;zk2:2181;zk3:2181/bkjournal
-
- The final part /bkjournal specifies the znode in zookeeper where
- ledger metadata will be store. Administrators can set this to anything
- they wish.
-
-

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/dev-support/findbugsExcludeFile.xml

@@ -1,5 +0,0 @@
-<FindBugsFilter>
-     <Match>
-       <Class name="~org.apache.hadoop.contrib.bkjournal.BKJournalProtos.*" />
-     </Match>
-</FindBugsFilter>

+ 0 - 175
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml

@@ -1,175 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
-http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-alpha2-SNAPSHOT</version>
-    <relativePath>../../../../../hadoop-project</relativePath>
-  </parent>
-
-  <groupId>org.apache.hadoop.contrib</groupId>
-  <artifactId>hadoop-hdfs-bkjournal</artifactId>
-  <version>3.0.0-alpha2-SNAPSHOT</version>
-  <description>Apache Hadoop HDFS BookKeeper Journal</description>
-  <name>Apache Hadoop HDFS BookKeeper Journal</name>
-  <packaging>jar</packaging>
-
-  <properties>
-    <hadoop.component>hdfs</hadoop.component>
-    <hadoop.common.build.dir>${basedir}/../../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency> 
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency> 
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency> 
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.bookkeeper</groupId>
-      <artifactId>bookkeeper-server</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/../../../../../hadoop-common-project/hadoop-common/src/main/proto</param>
-                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto</param>
-                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs/src/main/proto</param>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>bkjournal.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <configuration>
-          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-          <excludes>
-            <exclude>dev-support/findbugsExcludeFile.xml</exclude>
-          </excludes>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-  <profiles>
-    <profile>
-      <id>dist</id>
-      <build>
-        <plugins>
-          <plugin>
-            <artifactId>maven-dependency-plugin</artifactId>
-            <version>2.8</version>
-            <executions>
-              <execution>
-                <id>dist</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>copy</goal>
-                </goals>
-                <configuration>
-                  <artifactItems>
-                    <artifactItem>
-                      <groupId>org.apache.bookkeeper</groupId>
-                      <artifactId>bookkeeper-server</artifactId>
-                      <type>jar</type>
-                    </artifactItem>
-                  </artifactItems>
-                  <outputDirectory>${project.build.directory}/lib</outputDirectory>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>

+ 0 - 264
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java

@@ -1,264 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Enumeration;
-
-import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.BKException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * Input stream which reads from a BookKeeper ledger.
- */
-class BookKeeperEditLogInputStream extends EditLogInputStream {
-  static final Log LOG = LogFactory.getLog(BookKeeperEditLogInputStream.class);
-
-  private final long firstTxId;
-  private final long lastTxId;
-  private final int logVersion;
-  private final boolean inProgress;
-  private final LedgerHandle lh;
-
-  private final FSEditLogOp.Reader reader;
-  private final FSEditLogLoader.PositionTrackingInputStream tracker;
-
-  /**
-   * Construct BookKeeper edit log input stream.
-   * Starts reading from the first entry of the ledger.
-   */
-  BookKeeperEditLogInputStream(final LedgerHandle lh, 
-                               final EditLogLedgerMetadata metadata)
-      throws IOException {
-    this(lh, metadata, 0);
-  }
-
-  /**
-   * Construct BookKeeper edit log input stream. 
-   * Starts reading from firstBookKeeperEntry. This allows the stream
-   * to take a shortcut during recovery, as it doesn't have to read
-   * every edit log transaction to find out what the last one is.
-   */
-  BookKeeperEditLogInputStream(LedgerHandle lh, EditLogLedgerMetadata metadata,
-                               long firstBookKeeperEntry) 
-      throws IOException {
-    this.lh = lh;
-    this.firstTxId = metadata.getFirstTxId();
-    this.lastTxId = metadata.getLastTxId();
-    this.logVersion = metadata.getDataLayoutVersion();
-    this.inProgress = metadata.isInProgress();
-
-    if (firstBookKeeperEntry < 0
-        || firstBookKeeperEntry > lh.getLastAddConfirmed()) {
-      throw new IOException("Invalid first bk entry to read: "
-          + firstBookKeeperEntry + ", LAC: " + lh.getLastAddConfirmed());
-    }
-    BufferedInputStream bin = new BufferedInputStream(
-        new LedgerInputStream(lh, firstBookKeeperEntry));
-    tracker = new FSEditLogLoader.PositionTrackingInputStream(bin);
-    DataInputStream in = new DataInputStream(tracker);
-
-    reader = FSEditLogOp.Reader.create(in, tracker, logVersion);
-  }
-
-  @Override
-  public long getFirstTxId() {
-    return firstTxId;
-  }
-
-  @Override
-  public long getLastTxId() {
-    return lastTxId;
-  }
-  
-  @Override
-  public int getVersion(boolean verifyVersion) throws IOException {
-    return logVersion;
-  }
-
-  @Override
-  protected FSEditLogOp nextOp() throws IOException {
-    return reader.readOp(false);
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      lh.close();
-    } catch (BKException e) {
-      throw new IOException("Exception closing ledger", e);
-    } catch (InterruptedException e) {
-      throw new IOException("Interrupted closing ledger", e);
-    }
-  }
-
-  @Override
-  public long getPosition() {
-    return tracker.getPos();
-  }
-
-  @Override
-  public long length() throws IOException {
-    return lh.getLength();
-  }
-  
-  @Override
-  public String getName() {
-    return String.format(
-        "BookKeeperLedger[ledgerId=%d,firstTxId=%d,lastTxId=%d]", lh.getId(),
-        firstTxId, lastTxId);
-  }
-
-  @Override
-  public boolean isInProgress() {
-    return inProgress;
-  }
-
-  /**
-   * Skip forward to specified transaction id.
-   * Currently we do this by just iterating forward.
-   * If this proves to be too expensive, this can be reimplemented
-   * with a binary search over bk entries
-   */
-  public void skipTo(long txId) throws IOException {
-    long numToSkip = getFirstTxId() - txId;
-
-    FSEditLogOp op = null;
-    for (long i = 0; i < numToSkip; i++) {
-      op = readOp();
-    }
-    if (op != null && op.getTransactionId() != txId-1) {
-      throw new IOException("Corrupt stream, expected txid "
-          + (txId-1) + ", got " + op.getTransactionId());
-    }
-  }
-
-  @Override
-  public String toString() {
-    return ("BookKeeperEditLogInputStream {" + this.getName() + "}");
-  }
-
-  @Override
-  public void setMaxOpSize(int maxOpSize) {
-    reader.setMaxOpSize(maxOpSize);
-  }
-
-  @Override
-  public boolean isLocalLog() {
-    return false;
-  }
-
-  /**
-   * Input stream implementation which can be used by 
-   * FSEditLogOp.Reader
-   */
-  private static class LedgerInputStream extends InputStream {
-    private long readEntries;
-    private InputStream entryStream = null;
-    private final LedgerHandle lh;
-    private final long maxEntry;
-
-    /**
-     * Construct ledger input stream
-     * @param lh the ledger handle to read from
-     * @param firstBookKeeperEntry ledger entry to start reading from
-     */
-    LedgerInputStream(LedgerHandle lh, long firstBookKeeperEntry) 
-        throws IOException {
-      this.lh = lh;
-      readEntries = firstBookKeeperEntry;
-
-      maxEntry = lh.getLastAddConfirmed();
-    }
-
-    /**
-     * Get input stream representing next entry in the
-     * ledger.
-     * @return input stream, or null if no more entries
-     */
-    private InputStream nextStream() throws IOException {
-      try {        
-        if (readEntries > maxEntry) {
-          return null;
-        }
-        Enumeration<LedgerEntry> entries 
-          = lh.readEntries(readEntries, readEntries);
-        readEntries++;
-        if (entries.hasMoreElements()) {
-            LedgerEntry e = entries.nextElement();
-            assert !entries.hasMoreElements();
-            return e.getEntryInputStream();
-        }
-      } catch (BKException e) {
-        throw new IOException("Error reading entries from bookkeeper", e);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted reading entries from bookkeeper", e);
-      }
-      return null;
-    }
-
-    @Override
-    public int read() throws IOException {
-      byte[] b = new byte[1];
-      if (read(b, 0, 1) != 1) {
-        return -1;
-      } else {
-        return b[0];
-      }
-    }
-    
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-      try {
-        int read = 0;
-        if (entryStream == null) {
-          entryStream = nextStream();
-          if (entryStream == null) {
-            return read;
-          }
-        }
-
-        while (read < len) {
-          int thisread = entryStream.read(b, off+read, (len-read));
-          if (thisread == -1) {
-            entryStream = nextStream();
-            if (entryStream == null) {
-              return read;
-            }
-          } else {
-            read += thisread;
-          }
-        }
-        return read;
-      } catch (IOException e) {
-        throw e;
-      }
-
-    }
-  }
-}

+ 0 - 188
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java

@@ -1,188 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.CountDownLatch;
-
-import java.util.Arrays;
-
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
-
-import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.io.DataOutputBuffer;
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * Output stream for BookKeeper Journal.
- * Multiple complete edit log entries are packed into a single bookkeeper
- * entry before sending it over the network. The fact that the edit log entries
- * are complete in the bookkeeper entries means that each bookkeeper log entry
- *can be read as a complete edit log. This is useful for recover, as we don't
- * need to read through the entire edit log segment to get the last written
- * entry.
- */
-class BookKeeperEditLogOutputStream
-  extends EditLogOutputStream implements AddCallback {
-  static final Log LOG = LogFactory.getLog(BookKeeperEditLogOutputStream.class);
-
-  private final DataOutputBuffer bufCurrent;
-  private final AtomicInteger outstandingRequests;
-  private final int transmissionThreshold;
-  private final LedgerHandle lh;
-  private CountDownLatch syncLatch;
-  private final AtomicInteger transmitResult
-    = new AtomicInteger(BKException.Code.OK);
-  private final Writer writer;
-
-  /**
-   * Construct an edit log output stream which writes to a ledger.
-
-   */
-  protected BookKeeperEditLogOutputStream(Configuration conf, LedgerHandle lh)
-      throws IOException {
-    super();
-
-    bufCurrent = new DataOutputBuffer();
-    outstandingRequests = new AtomicInteger(0);
-    syncLatch = null;
-    this.lh = lh;
-    this.writer = new Writer(bufCurrent);
-    this.transmissionThreshold
-      = conf.getInt(BookKeeperJournalManager.BKJM_OUTPUT_BUFFER_SIZE,
-                    BookKeeperJournalManager.BKJM_OUTPUT_BUFFER_SIZE_DEFAULT);
-  }
-
-  @Override
-  public void create(int layoutVersion) throws IOException {
-    // noop
-  }
-
-  @Override
-  public void close() throws IOException {
-    setReadyToFlush();
-    flushAndSync(true);
-    try {
-      lh.close();
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted waiting on close", ie);
-    } catch (BKException bke) {
-      throw new IOException("BookKeeper error during close", bke);
-    }
-  }
-
-  @Override
-  public void abort() throws IOException {
-    try {
-      lh.close();
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted waiting on close", ie);
-    } catch (BKException bke) {
-      throw new IOException("BookKeeper error during abort", bke);
-    }
-
-  }
-
-  @Override
-  public void writeRaw(final byte[] data, int off, int len) throws IOException {
-    throw new IOException("Not supported for BK");
-  }
-
-  @Override
-  public void write(FSEditLogOp op) throws IOException {
-    writer.writeOp(op);
-
-    if (bufCurrent.getLength() > transmissionThreshold) {
-      transmit();
-    }
-  }
-
-  @Override
-  public void setReadyToFlush() throws IOException {
-    transmit();
-
-    synchronized (this) {
-      syncLatch = new CountDownLatch(outstandingRequests.get());
-    }
-  }
-
-  @Override
-  public void flushAndSync(boolean durable) throws IOException {
-    assert(syncLatch != null);
-    try {
-      syncLatch.await();
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted waiting on latch", ie);
-    }
-    if (transmitResult.get() != BKException.Code.OK) {
-      throw new IOException("Failed to write to bookkeeper; Error is ("
-                            + transmitResult.get() + ") "
-                            + BKException.getMessage(transmitResult.get()));
-    }
-
-    syncLatch = null;
-    // wait for whatever we wait on
-  }
-
-  /**
-   * Transmit the current buffer to bookkeeper.
-   * Synchronised at the FSEditLog level. #write() and #setReadyToFlush()
-   * are never called at the same time.
-   */
-  private void transmit() throws IOException {
-    if (!transmitResult.compareAndSet(BKException.Code.OK,
-                                     BKException.Code.OK)) {
-      throw new IOException("Trying to write to an errored stream;"
-          + " Error code : (" + transmitResult.get()
-          + ") " + BKException.getMessage(transmitResult.get()));
-    }
-    if (bufCurrent.getLength() > 0) {
-      byte[] entry = Arrays.copyOf(bufCurrent.getData(),
-                                   bufCurrent.getLength());
-      lh.asyncAddEntry(entry, this, null);
-      bufCurrent.reset();
-      outstandingRequests.incrementAndGet();
-    }
-  }
-
-  @Override
-  public void addComplete(int rc, LedgerHandle handle,
-                          long entryId, Object ctx) {
-    synchronized(this) {
-      outstandingRequests.decrementAndGet();
-      if (!transmitResult.compareAndSet(BKException.Code.OK, rc)) {
-        LOG.warn("Tried to set transmit result to (" + rc + ") \""
-            + BKException.getMessage(rc) + "\""
-            + " but is already (" + transmitResult.get() + ") \""
-            + BKException.getMessage(transmitResult.get()) + "\"");
-      }
-      CountDownLatch l = syncLatch;
-      if (l != null) {
-        l.countDown();
-      }
-    }
-  }
-}

+ 0 - 893
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -1,893 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.namenode.JournalManager;
-import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
-import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.conf.Configuration;
-
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.util.ZkUtils;
-
-import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.AsyncCallback.StringCallback;
-import org.apache.zookeeper.ZKUtil;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.io.IOException;
-
-import java.net.URI;
-
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.VersionProto;
-import com.google.protobuf.TextFormat;
-import static com.google.common.base.Charsets.UTF_8;
-
-import org.apache.commons.io.Charsets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import com.google.common.annotations.VisibleForTesting;
-/**
- * BookKeeper Journal Manager
- *
- * To use, add the following to hdfs-site.xml.
- * <pre>
- * {@code
- * <property>
- *   <name>dfs.namenode.edits.dir</name>
- *   <value>bookkeeper://zk1:2181;zk2:2181;zk3:2181/hdfsjournal</value>
- * </property>
- *
- * <property>
- *   <name>dfs.namenode.edits.journal-plugin.bookkeeper</name>
- *   <value>org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager</value>
- * </property>
- * }
- * </pre>
- * The URI format for bookkeeper is bookkeeper://[zkEnsemble]/[rootZnode]
- * [zookkeeper ensemble] is a list of semi-colon separated, zookeeper host:port
- * pairs. In the example above there are 3 servers, in the ensemble,
- * zk1, zk2 &amp; zk3, each one listening on port 2181.
- *
- * [root znode] is the path of the zookeeper znode, under which the editlog
- * information will be stored.
- *
- * Other configuration options are:
- * <ul>
- *   <li><b>dfs.namenode.bookkeeperjournal.output-buffer-size</b>
- *       Number of bytes a bookkeeper journal stream will buffer before
- *       forcing a flush. Default is 1024.</li>
- *   <li><b>dfs.namenode.bookkeeperjournal.ensemble-size</b>
- *       Number of bookkeeper servers in edit log ledger ensembles. This
- *       is the number of bookkeeper servers which need to be available
- *       for the ledger to be writable. Default is 3.</li>
- *   <li><b>dfs.namenode.bookkeeperjournal.quorum-size</b>
- *       Number of bookkeeper servers in the write quorum. This is the
- *       number of bookkeeper servers which must have acknowledged the
- *       write of an entry before it is considered written.
- *       Default is 2.</li>
- *   <li><b>dfs.namenode.bookkeeperjournal.digestPw</b>
- *       Password to use when creating ledgers. </li>
- *   <li><b>dfs.namenode.bookkeeperjournal.zk.session.timeout</b>
- *       Session timeout for Zookeeper client from BookKeeper Journal Manager.
- *       Hadoop recommends that, this value should be less than the ZKFC 
- *       session timeout value. Default value is 3000.</li>
- * </ul>
- */
-public class BookKeeperJournalManager implements JournalManager {
-  static final Log LOG = LogFactory.getLog(BookKeeperJournalManager.class);
-
-  public static final String BKJM_OUTPUT_BUFFER_SIZE
-    = "dfs.namenode.bookkeeperjournal.output-buffer-size";
-  public static final int BKJM_OUTPUT_BUFFER_SIZE_DEFAULT = 1024;
-
-  public static final String BKJM_BOOKKEEPER_ENSEMBLE_SIZE
-    = "dfs.namenode.bookkeeperjournal.ensemble-size";
-  public static final int BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT = 3;
-
- public static final String BKJM_BOOKKEEPER_QUORUM_SIZE
-    = "dfs.namenode.bookkeeperjournal.quorum-size";
-  public static final int BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT = 2;
-
-  public static final String BKJM_BOOKKEEPER_DIGEST_PW
-    = "dfs.namenode.bookkeeperjournal.digestPw";
-  public static final String BKJM_BOOKKEEPER_DIGEST_PW_DEFAULT = "";
-
-  private static final int BKJM_LAYOUT_VERSION = -1;
-  
-  public static final String BKJM_ZK_SESSION_TIMEOUT 
-    = "dfs.namenode.bookkeeperjournal.zk.session.timeout";
-  public static final int BKJM_ZK_SESSION_TIMEOUT_DEFAULT = 3000;
-
-  private static final String BKJM_EDIT_INPROGRESS = "inprogress_";
-
-  public static final String BKJM_ZK_LEDGERS_AVAILABLE_PATH
-    = "dfs.namenode.bookkeeperjournal.zk.availablebookies";
-
-  public static final String BKJM_ZK_LEDGERS_AVAILABLE_PATH_DEFAULT
-    = "/ledgers/available";
-
-  public static final String BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS
-    = "dfs.namenode.bookkeeperjournal.speculativeReadTimeoutMs";
-  public static final int BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT
-    = 2000;
-
-  public static final String BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_SEC
-    = "dfs.namenode.bookkeeperjournal.readEntryTimeoutSec";
-  public static final int BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT = 5;
-
-  public static final String BKJM_BOOKKEEPER_ACK_QUORUM_SIZE 
-    = "dfs.namenode.bookkeeperjournal.ack.quorum-size";
-
-  public static final String BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC
-    = "dfs.namenode.bookkeeperjournal.addEntryTimeoutSec";
-  public static final int BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT = 5;
-
-  private ZooKeeper zkc;
-  private final Configuration conf;
-  private final BookKeeper bkc;
-  private final CurrentInprogress ci;
-  private final String basePath;
-  private final String ledgerPath;
-  private final String versionPath;
-  private final MaxTxId maxTxId;
-  private final int ensembleSize;
-  private final int quorumSize;
-  private final int ackQuorumSize;
-  private final int addEntryTimeout;
-  private final String digestpw;
-  private final int speculativeReadTimeout;
-  private final int readEntryTimeout;
-  private final CountDownLatch zkConnectLatch;
-  private final NamespaceInfo nsInfo;
-  private boolean initialized = false;
-  private LedgerHandle currentLedger = null;
-
-  /**
-   * Construct a Bookkeeper journal manager.
-   */
-  public BookKeeperJournalManager(Configuration conf, URI uri,
-      NamespaceInfo nsInfo) throws IOException {
-    this.conf = conf;
-    this.nsInfo = nsInfo;
-
-    String zkConnect = uri.getAuthority().replace(";", ",");
-    basePath = uri.getPath();
-    ensembleSize = conf.getInt(BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-                               BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT);
-    quorumSize = conf.getInt(BKJM_BOOKKEEPER_QUORUM_SIZE,
-                             BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT);
-    ackQuorumSize = conf.getInt(BKJM_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
-    addEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
-                             BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT);
-    speculativeReadTimeout = conf.getInt(
-                             BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
-                             BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT);
-    readEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_SEC,
-                             BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT);
-
-    ledgerPath = basePath + "/ledgers";
-    String maxTxIdPath = basePath + "/maxtxid";
-    String currentInprogressNodePath = basePath + "/CurrentInprogress";
-    versionPath = basePath + "/version";
-    digestpw = conf.get(BKJM_BOOKKEEPER_DIGEST_PW,
-                        BKJM_BOOKKEEPER_DIGEST_PW_DEFAULT);
-
-    try {
-      zkConnectLatch = new CountDownLatch(1);
-      int bkjmZKSessionTimeout = conf.getInt(BKJM_ZK_SESSION_TIMEOUT,
-          BKJM_ZK_SESSION_TIMEOUT_DEFAULT);
-      zkc = new ZooKeeper(zkConnect, bkjmZKSessionTimeout,
-          new ZkConnectionWatcher());
-      // Configured zk session timeout + some extra grace period (here
-      // BKJM_ZK_SESSION_TIMEOUT_DEFAULT used as grace period)
-      int zkConnectionLatchTimeout = bkjmZKSessionTimeout
-          + BKJM_ZK_SESSION_TIMEOUT_DEFAULT;
-      if (!zkConnectLatch
-          .await(zkConnectionLatchTimeout, TimeUnit.MILLISECONDS)) {
-        throw new IOException("Error connecting to zookeeper");
-      }
-
-      prepareBookKeeperEnv();
-      ClientConfiguration clientConf = new ClientConfiguration();
-      clientConf.setSpeculativeReadTimeout(speculativeReadTimeout);
-      clientConf.setReadEntryTimeout(readEntryTimeout);
-      clientConf.setAddEntryTimeout(addEntryTimeout);
-      bkc = new BookKeeper(clientConf, zkc);
-    } catch (KeeperException e) {
-      throw new IOException("Error initializing zk", e);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted while initializing bk journal manager",
-                            ie);
-    }
-
-    ci = new CurrentInprogress(zkc, currentInprogressNodePath);
-    maxTxId = new MaxTxId(zkc, maxTxIdPath);
-  }
-
-  /**
-   * Pre-creating bookkeeper metadata path in zookeeper.
-   */
-  private void prepareBookKeeperEnv() throws IOException {
-    // create bookie available path in zookeeper if it doesn't exists
-    final String zkAvailablePath = conf.get(BKJM_ZK_LEDGERS_AVAILABLE_PATH,
-        BKJM_ZK_LEDGERS_AVAILABLE_PATH_DEFAULT);
-    final CountDownLatch zkPathLatch = new CountDownLatch(1);
-
-    final AtomicBoolean success = new AtomicBoolean(false);
-    StringCallback callback = new StringCallback() {
-      @Override
-      public void processResult(int rc, String path, Object ctx, String name) {
-        if (KeeperException.Code.OK.intValue() == rc
-            || KeeperException.Code.NODEEXISTS.intValue() == rc) {
-          LOG.info("Successfully created bookie available path : "
-              + zkAvailablePath);
-          success.set(true);
-        } else {
-          KeeperException.Code code = KeeperException.Code.get(rc);
-          LOG.error("Error : "
-                  + KeeperException.create(code, path).getMessage()
-                  + ", failed to create bookie available path : "
-                  + zkAvailablePath);
-        }
-        zkPathLatch.countDown();
-      }
-    };
-    ZkUtils.asyncCreateFullPathOptimistic(zkc, zkAvailablePath, new byte[0],
-        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, callback, null);
-
-    try {
-      if (!zkPathLatch.await(zkc.getSessionTimeout(), TimeUnit.MILLISECONDS)
-          || !success.get()) {
-        throw new IOException("Couldn't create bookie available path :"
-            + zkAvailablePath + ", timed out " + zkc.getSessionTimeout()
-            + " millis");
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new IOException(
-          "Interrupted when creating the bookie available path : "
-              + zkAvailablePath, e);
-    }
-  }
-
-  @Override
-  public void format(NamespaceInfo ns) throws IOException {
-    try {
-      // delete old info
-      Stat baseStat = null;
-      Stat ledgerStat = null;
-      if ((baseStat = zkc.exists(basePath, false)) != null) {
-        if ((ledgerStat = zkc.exists(ledgerPath, false)) != null) {
-          for (EditLogLedgerMetadata l : getLedgerList(true)) {
-            try {
-              bkc.deleteLedger(l.getLedgerId());
-            } catch (BKException.BKNoSuchLedgerExistsException bke) {
-              LOG.warn("Ledger " + l.getLedgerId() + " does not exist;"
-                       + " Cannot delete.");
-            }
-          }
-        }
-        ZKUtil.deleteRecursive(zkc, basePath);
-      }
-
-      // should be clean now.
-      zkc.create(basePath, new byte[] {'0'},
-          Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
-      VersionProto.Builder builder = VersionProto.newBuilder();
-      builder.setNamespaceInfo(PBHelper.convert(ns))
-        .setLayoutVersion(BKJM_LAYOUT_VERSION);
-
-      byte[] data = TextFormat.printToString(builder.build()).getBytes(UTF_8);
-      zkc.create(versionPath, data,
-                 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
-      zkc.create(ledgerPath, new byte[] {'0'},
-                 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    } catch (KeeperException ke) {
-      LOG.error("Error accessing zookeeper to format", ke);
-      throw new IOException("Error accessing zookeeper to format", ke);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted during format", ie);
-    } catch (BKException bke) {
-      throw new IOException("Error cleaning up ledgers during format", bke);
-    }
-  }
-  
-  @Override
-  public boolean hasSomeData() throws IOException {
-    try {
-      return zkc.exists(basePath, false) != null;
-    } catch (KeeperException ke) {
-      throw new IOException("Couldn't contact zookeeper", ke);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted while checking for data", ie);
-    }
-  }
-
-  synchronized private void checkEnv() throws IOException {
-    if (!initialized) {
-      try {
-        Stat versionStat = zkc.exists(versionPath, false);
-        if (versionStat == null) {
-          throw new IOException("Environment not initialized. "
-                                +"Have you forgotten to format?");
-        }
-        byte[] d = zkc.getData(versionPath, false, versionStat);
-
-        VersionProto.Builder builder = VersionProto.newBuilder();
-        TextFormat.merge(new String(d, UTF_8), builder);
-        if (!builder.isInitialized()) {
-          throw new IOException("Invalid/Incomplete data in znode");
-        }
-        VersionProto vp = builder.build();
-
-        // There's only one version at the moment
-        assert vp.getLayoutVersion() == BKJM_LAYOUT_VERSION;
-
-        NamespaceInfo readns = PBHelper.convert(vp.getNamespaceInfo());
-
-        if (nsInfo.getNamespaceID() != readns.getNamespaceID() ||
-            !nsInfo.clusterID.equals(readns.getClusterID()) ||
-            !nsInfo.getBlockPoolID().equals(readns.getBlockPoolID())) {
-          String err = String.format("Environment mismatch. Running process %s"
-                                     +", stored in ZK %s", nsInfo, readns);
-          LOG.error(err);
-          throw new IOException(err);
-        }
-
-        ci.init();
-        initialized = true;
-      } catch (KeeperException ke) {
-        throw new IOException("Cannot access ZooKeeper", ke);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Interrupted while checking environment", ie);
-      }
-    }
-  }
-
-  /**
-   * Start a new log segment in a BookKeeper ledger.
-   * First ensure that we have the write lock for this journal.
-   * Then create a ledger and stream based on that ledger.
-   * The ledger id is written to the inprogress znode, so that in the
-   * case of a crash, a recovery process can find the ledger we were writing
-   * to when we crashed.
-   * @param txId First transaction id to be written to the stream
-   */
-  @Override
-  public EditLogOutputStream startLogSegment(long txId, int layoutVersion)
-      throws IOException {
-    checkEnv();
-
-    if (txId <= maxTxId.get()) {
-      throw new IOException("We've already seen " + txId
-          + ". A new stream cannot be created with it");
-    }
-
-    try {
-      String existingInprogressNode = ci.read();
-      if (null != existingInprogressNode
-          && zkc.exists(existingInprogressNode, false) != null) {
-        throw new IOException("Inprogress node already exists");
-      }
-      if (currentLedger != null) {
-        // bookkeeper errored on last stream, clean up ledger
-        currentLedger.close();
-      }
-      currentLedger = bkc.createLedger(ensembleSize, quorumSize, ackQuorumSize,
-                                       BookKeeper.DigestType.MAC,
-                                       digestpw.getBytes(Charsets.UTF_8));
-    } catch (BKException bke) {
-      throw new IOException("Error creating ledger", bke);
-    } catch (KeeperException ke) {
-      throw new IOException("Error in zookeeper while creating ledger", ke);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted creating ledger", ie);
-    }
-
-    try {
-      String znodePath = inprogressZNode(txId);
-      EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
-          layoutVersion, currentLedger.getId(), txId);
-      /* Write the ledger metadata out to the inprogress ledger znode
-       * This can fail if for some reason our write lock has
-       * expired (@see WriteLock) and another process has managed to
-       * create the inprogress znode.
-       * In this case, throw an exception. We don't want to continue
-       * as this would lead to a split brain situation.
-       */
-      l.write(zkc, znodePath);
-
-      maxTxId.store(txId);
-      ci.update(znodePath);
-      return new BookKeeperEditLogOutputStream(conf, currentLedger);
-    } catch (KeeperException ke) {
-      cleanupLedger(currentLedger);
-      throw new IOException("Error storing ledger metadata", ke);
-    }
-  }
-
-  private void cleanupLedger(LedgerHandle lh) {
-    try {
-      long id = currentLedger.getId();
-      currentLedger.close();
-      bkc.deleteLedger(id);
-    } catch (BKException bke) {
-      //log & ignore, an IOException will be thrown soon
-      LOG.error("Error closing ledger", bke);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      LOG.warn("Interrupted while closing ledger", ie);
-    }
-  }
-
-
-
-  /**
-   * Finalize a log segment. If the journal manager is currently
-   * writing to a ledger, ensure that this is the ledger of the log segment
-   * being finalized.
-   *
-   * Otherwise this is the recovery case. In the recovery case, ensure that
-   * the firstTxId of the ledger matches firstTxId for the segment we are
-   * trying to finalize.
-   */
-  @Override
-  public void finalizeLogSegment(long firstTxId, long lastTxId)
-      throws IOException {
-    checkEnv();
-
-    String inprogressPath = inprogressZNode(firstTxId);
-    try {
-      Stat inprogressStat = zkc.exists(inprogressPath, false);
-      if (inprogressStat == null) {
-        throw new IOException("Inprogress znode " + inprogressPath
-                              + " doesn't exist");
-      }
-
-      EditLogLedgerMetadata l
-        =  EditLogLedgerMetadata.read(zkc, inprogressPath);
-
-      if (currentLedger != null) { // normal, non-recovery case
-        if (l.getLedgerId() == currentLedger.getId()) {
-          try {
-            currentLedger.close();
-          } catch (BKException bke) {
-            LOG.error("Error closing current ledger", bke);
-          }
-          currentLedger = null;
-        } else {
-          throw new IOException(
-              "Active ledger has different ID to inprogress. "
-              + l.getLedgerId() + " found, "
-              + currentLedger.getId() + " expected");
-        }
-      }
-
-      if (l.getFirstTxId() != firstTxId) {
-        throw new IOException("Transaction id not as expected, "
-            + l.getFirstTxId() + " found, " + firstTxId + " expected");
-      }
-
-      l.finalizeLedger(lastTxId);
-      String finalisedPath = finalizedLedgerZNode(firstTxId, lastTxId);
-      try {
-        l.write(zkc, finalisedPath);
-      } catch (KeeperException.NodeExistsException nee) {
-        if (!l.verify(zkc, finalisedPath)) {
-          throw new IOException("Node " + finalisedPath + " already exists"
-                                + " but data doesn't match");
-        }
-      }
-      maxTxId.store(lastTxId);
-      zkc.delete(inprogressPath, inprogressStat.getVersion());
-      String inprogressPathFromCI = ci.read();
-      if (inprogressPath.equals(inprogressPathFromCI)) {
-        ci.clear();
-      }
-    } catch (KeeperException e) {
-      throw new IOException("Error finalising ledger", e);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Error finalising ledger", ie);
-    }
-  }
-
-  public void selectInputStreams(
-      Collection<EditLogInputStream> streams,
-      long fromTxnId, boolean inProgressOk) throws IOException {
-    selectInputStreams(streams, fromTxnId, inProgressOk, false);
-  }
-
-  @Override
-  public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean onlyDurableTxns)
-      throws IOException {
-    List<EditLogLedgerMetadata> currentLedgerList = getLedgerList(fromTxId,
-        inProgressOk);
-    try {
-      BookKeeperEditLogInputStream elis = null;
-      for (EditLogLedgerMetadata l : currentLedgerList) {
-        long lastTxId = l.getLastTxId();
-        if (l.isInProgress()) {
-          lastTxId = recoverLastTxId(l, false);
-        }
-        // Check once again, required in case of InProgress and is case of any
-        // gap.
-        if (fromTxId >= l.getFirstTxId() && fromTxId <= lastTxId) {
-          LedgerHandle h;
-          if (l.isInProgress()) { // we don't want to fence the current journal
-            h = bkc.openLedgerNoRecovery(l.getLedgerId(),
-                BookKeeper.DigestType.MAC, digestpw.getBytes(Charsets.UTF_8));
-          } else {
-            h = bkc.openLedger(l.getLedgerId(), BookKeeper.DigestType.MAC,
-                digestpw.getBytes(Charsets.UTF_8));
-          }
-          elis = new BookKeeperEditLogInputStream(h, l);
-          elis.skipTo(fromTxId);
-        } else {
-          // If mismatches then there might be some gap, so we should not check
-          // further.
-          return;
-        }
-        streams.add(elis);
-        if (elis.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
-          return;
-        }
-        fromTxId = elis.getLastTxId() + 1;
-      }
-    } catch (BKException e) {
-      throw new IOException("Could not open ledger for " + fromTxId, e);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted opening ledger for " + fromTxId, ie);
-    }
-  }
-
-  long getNumberOfTransactions(long fromTxId, boolean inProgressOk)
-      throws IOException {
-    long count = 0;
-    long expectedStart = 0;
-    for (EditLogLedgerMetadata l : getLedgerList(inProgressOk)) {
-      long lastTxId = l.getLastTxId();
-      if (l.isInProgress()) {
-        lastTxId = recoverLastTxId(l, false);
-        if (lastTxId == HdfsServerConstants.INVALID_TXID) {
-          break;
-        }
-      }
-
-      assert lastTxId >= l.getFirstTxId();
-
-      if (lastTxId < fromTxId) {
-        continue;
-      } else if (l.getFirstTxId() <= fromTxId && lastTxId >= fromTxId) {
-        // we can start in the middle of a segment
-        count = (lastTxId - l.getFirstTxId()) + 1;
-        expectedStart = lastTxId + 1;
-      } else {
-        if (expectedStart != l.getFirstTxId()) {
-          if (count == 0) {
-            throw new CorruptionException("StartTxId " + l.getFirstTxId()
-                + " is not as expected " + expectedStart
-                + ". Gap in transaction log?");
-          } else {
-            break;
-          }
-        }
-        count += (lastTxId - l.getFirstTxId()) + 1;
-        expectedStart = lastTxId + 1;
-      }
-    }
-    return count;
-  }
-
-  @Override
-  public void recoverUnfinalizedSegments() throws IOException {
-    checkEnv();
-
-    synchronized (this) {
-      try {
-        List<String> children = zkc.getChildren(ledgerPath, false);
-        for (String child : children) {
-          if (!child.startsWith(BKJM_EDIT_INPROGRESS)) {
-            continue;
-          }
-          String znode = ledgerPath + "/" + child;
-          EditLogLedgerMetadata l = EditLogLedgerMetadata.read(zkc, znode);
-          try {
-            long endTxId = recoverLastTxId(l, true);
-            if (endTxId == HdfsServerConstants.INVALID_TXID) {
-              LOG.error("Unrecoverable corruption has occurred in segment "
-                  + l.toString() + " at path " + znode
-                  + ". Unable to continue recovery.");
-              throw new IOException("Unrecoverable corruption,"
-                  + " please check logs.");
-            }
-            finalizeLogSegment(l.getFirstTxId(), endTxId);
-          } catch (SegmentEmptyException see) {
-            LOG.warn("Inprogress znode " + child
-                + " refers to a ledger which is empty. This occurs when the NN"
-                + " crashes after opening a segment, but before writing the"
-                + " OP_START_LOG_SEGMENT op. It is safe to delete."
-                + " MetaData [" + l.toString() + "]");
-
-            // If the max seen transaction is the same as what would
-            // have been the first transaction of the failed ledger,
-            // decrement it, as that transaction never happened and as
-            // such, is _not_ the last seen
-            if (maxTxId.get() == l.getFirstTxId()) {
-              maxTxId.reset(maxTxId.get() - 1);
-            }
-
-            zkc.delete(znode, -1);
-          }
-        }
-      } catch (KeeperException.NoNodeException nne) {
-          // nothing to recover, ignore
-      } catch (KeeperException ke) {
-        throw new IOException("Couldn't get list of inprogress segments", ke);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Interrupted getting list of inprogress segments",
-                              ie);
-      }
-    }
-  }
-
-  @Override
-  public void purgeLogsOlderThan(long minTxIdToKeep)
-      throws IOException {
-    checkEnv();
-
-    for (EditLogLedgerMetadata l : getLedgerList(false)) {
-      if (l.getLastTxId() < minTxIdToKeep) {
-        try {
-          Stat stat = zkc.exists(l.getZkPath(), false);
-          zkc.delete(l.getZkPath(), stat.getVersion());
-          bkc.deleteLedger(l.getLedgerId());
-        } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          LOG.error("Interrupted while purging " + l, ie);
-        } catch (BKException bke) {
-          LOG.error("Couldn't delete ledger from bookkeeper", bke);
-        } catch (KeeperException ke) {
-          LOG.error("Error deleting ledger entry in zookeeper", ke);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void doPreUpgrade() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void doUpgrade(Storage storage) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getJournalCTime() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void doFinalize() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
-      int targetLayoutVersion) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void doRollback() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void discardSegments(long startTxId) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      bkc.close();
-      zkc.close();
-    } catch (BKException bke) {
-      throw new IOException("Couldn't close bookkeeper client", bke);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted while closing journal manager", ie);
-    }
-  }
-
-  /**
-   * Set the amount of memory that this stream should use to buffer edits.
-   * Setting this will only affect future output stream. Streams
-   * which have currently be created won't be affected.
-   */
-  @Override
-  public void setOutputBufferCapacity(int size) {
-    conf.getInt(BKJM_OUTPUT_BUFFER_SIZE, size);
-  }
-
-  /**
-   * Find the id of the last edit log transaction writen to a edit log
-   * ledger.
-   */
-  private long recoverLastTxId(EditLogLedgerMetadata l, boolean fence)
-      throws IOException, SegmentEmptyException {
-    LedgerHandle lh = null;
-    try {
-      if (fence) {
-        lh = bkc.openLedger(l.getLedgerId(),
-                            BookKeeper.DigestType.MAC,
-                            digestpw.getBytes(Charsets.UTF_8));
-      } else {
-        lh = bkc.openLedgerNoRecovery(l.getLedgerId(),
-                                      BookKeeper.DigestType.MAC,
-                                      digestpw.getBytes(Charsets.UTF_8));
-      }
-    } catch (BKException bke) {
-      throw new IOException("Exception opening ledger for " + l, bke);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted opening ledger for " + l, ie);
-    }
-
-    BookKeeperEditLogInputStream in = null;
-
-    try {
-      long lastAddConfirmed = lh.getLastAddConfirmed();
-      if (lastAddConfirmed == -1) {
-        throw new SegmentEmptyException();
-      }
-
-      in = new BookKeeperEditLogInputStream(lh, l, lastAddConfirmed);
-
-      long endTxId = HdfsServerConstants.INVALID_TXID;
-      FSEditLogOp op = in.readOp();
-      while (op != null) {
-        if (endTxId == HdfsServerConstants.INVALID_TXID
-            || op.getTransactionId() == endTxId+1) {
-          endTxId = op.getTransactionId();
-        }
-        op = in.readOp();
-      }
-      return endTxId;
-    } finally {
-      if (in != null) {
-        in.close();
-      }
-    }
-  }
-
-  /**
-   * Get a list of all segments in the journal.
-   */
-  List<EditLogLedgerMetadata> getLedgerList(boolean inProgressOk)
-      throws IOException {
-    return getLedgerList(-1, inProgressOk);
-  }
-
-  private List<EditLogLedgerMetadata> getLedgerList(long fromTxId,
-      boolean inProgressOk) throws IOException {
-    List<EditLogLedgerMetadata> ledgers
-      = new ArrayList<EditLogLedgerMetadata>();
-    try {
-      List<String> ledgerNames = zkc.getChildren(ledgerPath, false);
-      for (String ledgerName : ledgerNames) {
-        if (!inProgressOk && ledgerName.contains(BKJM_EDIT_INPROGRESS)) {
-          continue;
-        }
-        String legderMetadataPath = ledgerPath + "/" + ledgerName;
-        try {
-          EditLogLedgerMetadata editLogLedgerMetadata = EditLogLedgerMetadata
-              .read(zkc, legderMetadataPath);
-          if (editLogLedgerMetadata.getLastTxId() != HdfsServerConstants.INVALID_TXID
-              && editLogLedgerMetadata.getLastTxId() < fromTxId) {
-            // exclude already read closed edits, but include inprogress edits
-            // as this will be handled in caller
-            continue;
-          }
-          ledgers.add(editLogLedgerMetadata);
-        } catch (KeeperException.NoNodeException e) {
-          LOG.warn("ZNode: " + legderMetadataPath
-              + " might have finalized and deleted."
-              + " So ignoring NoNodeException.");
-        }
-      }
-    } catch (KeeperException e) {
-      throw new IOException("Exception reading ledger list from zk", e);
-    } catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      throw new IOException("Interrupted getting list of ledgers from zk", ie);
-    }
-
-    Collections.sort(ledgers, EditLogLedgerMetadata.COMPARATOR);
-    return ledgers;
-  }
-
-  /**
-   * Get the znode path for a finalize ledger
-   */
-  String finalizedLedgerZNode(long startTxId, long endTxId) {
-    return String.format("%s/edits_%018d_%018d",
-                         ledgerPath, startTxId, endTxId);
-  }
-
-  /**
-   * Get the znode path for the inprogressZNode
-   */
-  String inprogressZNode(long startTxid) {
-    return ledgerPath + "/inprogress_" + Long.toString(startTxid, 16);
-  }
-
-  @VisibleForTesting
-  void setZooKeeper(ZooKeeper zk) {
-    this.zkc = zk;
-  }
-
-  /**
-   * Simple watcher to notify when zookeeper has connected
-   */
-  private class ZkConnectionWatcher implements Watcher {
-    public void process(WatchedEvent event) {
-      if (Event.KeeperState.SyncConnected.equals(event.getState())) {
-        zkConnectLatch.countDown();
-      }
-    }
-  }
-  
-  private static class SegmentEmptyException extends IOException {
-  }
-}

+ 0 - 160
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/CurrentInprogress.java

@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.io.IOException;
-import java.net.InetAddress;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.data.Stat;
-
-import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.CurrentInprogressProto;
-import com.google.protobuf.TextFormat;
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Distributed write permission lock, using ZooKeeper. Read the version number
- * and return the current inprogress node path available in CurrentInprogress
- * path. If it exist, caller can treat that some other client already operating
- * on it. Then caller can take action. If there is no inprogress node exist,
- * then caller can treat that there is no client operating on it. Later same
- * caller should update the his newly created inprogress node path. At this
- * point, if some other activities done on this node, version number might
- * change, so update will fail. So, this read, update api will ensure that there
- * is only node can continue further after checking with CurrentInprogress.
- */
-
-class CurrentInprogress {
-  static final Log LOG = LogFactory.getLog(CurrentInprogress.class);
-
-  private final ZooKeeper zkc;
-  private final String currentInprogressNode;
-  private volatile int versionNumberForPermission = -1;
-  private final String hostName = InetAddress.getLocalHost().toString();
-
-  CurrentInprogress(ZooKeeper zkc, String lockpath) throws IOException {
-    this.currentInprogressNode = lockpath;
-    this.zkc = zkc;
-  }
-
-  void init() throws IOException {
-    try {
-      Stat isCurrentInprogressNodeExists = zkc.exists(currentInprogressNode,
-                                                      false);
-      if (isCurrentInprogressNodeExists == null) {
-        try {
-          zkc.create(currentInprogressNode, null, Ids.OPEN_ACL_UNSAFE,
-                     CreateMode.PERSISTENT);
-        } catch (NodeExistsException e) {
-          // Node might created by other process at the same time. Ignore it.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(currentInprogressNode + " already created by other process.",
-                      e);
-          }
-        }
-      }
-    } catch (KeeperException e) {
-      throw new IOException("Exception accessing Zookeeper", e);
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted accessing Zookeeper", ie);
-    }
-  }
-
-  /**
-   * Update the path with prepending version number and hostname
-   * 
-   * @param path
-   *          - to be updated in zookeeper
-   * @throws IOException
-   */
-  void update(String path) throws IOException {
-    CurrentInprogressProto.Builder builder = CurrentInprogressProto.newBuilder();
-    builder.setPath(path).setHostname(hostName);
-
-    String content = TextFormat.printToString(builder.build());
-
-    try {
-      zkc.setData(this.currentInprogressNode, content.getBytes(UTF_8),
-          this.versionNumberForPermission);
-    } catch (KeeperException e) {
-      throw new IOException("Exception when setting the data "
-          + "[" + content + "] to CurrentInprogress. ", e);
-    } catch (InterruptedException e) {
-      throw new IOException("Interrupted while setting the data "
-          + "[" + content + "] to CurrentInprogress", e);
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Updated data[" + content + "] to CurrentInprogress");
-    }
-  }
-
-  /**
-   * Read the CurrentInprogress node data from Zookeeper and also get the znode
-   * version number. Return the 3rd field from the data. i.e saved path with
-   * #update api
-   * 
-   * @return available inprogress node path. returns null if not available.
-   * @throws IOException
-   */
-  String read() throws IOException {
-    Stat stat = new Stat();
-    byte[] data = null;
-    try {
-      data = zkc.getData(this.currentInprogressNode, false, stat);
-    } catch (KeeperException e) {
-      throw new IOException("Exception while reading the data from "
-          + currentInprogressNode, e);
-    } catch (InterruptedException e) {
-      throw new IOException("Interrupted while reading data from "
-          + currentInprogressNode, e);
-    }
-    this.versionNumberForPermission = stat.getVersion();
-    if (data != null) {
-      CurrentInprogressProto.Builder builder = CurrentInprogressProto.newBuilder();
-      TextFormat.merge(new String(data, UTF_8), builder);
-      if (!builder.isInitialized()) {
-        throw new IOException("Invalid/Incomplete data in znode");
-      }
-      return builder.build().getPath();
-    } else {
-      LOG.debug("No data available in CurrentInprogress");
-    }
-    return null;
-  }
-
-  /** Clear the CurrentInprogress node data */
-  void clear() throws IOException {
-    try {
-      zkc.setData(this.currentInprogressNode, null, versionNumberForPermission);
-    } catch (KeeperException e) {
-      throw new IOException(
-          "Exception when setting the data to CurrentInprogress node", e);
-    } catch (InterruptedException e) {
-      throw new IOException(
-          "Interrupted when setting the data to CurrentInprogress node", e);
-    }
-    LOG.debug("Cleared the data from CurrentInprogress");
-  }
-
-}

+ 0 - 217
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java

@@ -1,217 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.io.IOException;
-import java.util.Comparator;
-
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.KeeperException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.EditLogLedgerProto;
-import com.google.protobuf.TextFormat;
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Utility class for storing the metadata associated 
- * with a single edit log segment, stored in a single ledger
- */
-public class EditLogLedgerMetadata {
-  static final Log LOG = LogFactory.getLog(EditLogLedgerMetadata.class);
-
-  private String zkPath;
-  private final int dataLayoutVersion;
-  private final long ledgerId;
-  private final long firstTxId;
-  private long lastTxId;
-  private boolean inprogress;
-  
-  public static final Comparator COMPARATOR 
-    = new Comparator<EditLogLedgerMetadata>() {
-    public int compare(EditLogLedgerMetadata o1,
-        EditLogLedgerMetadata o2) {
-      if (o1.firstTxId < o2.firstTxId) {
-        return -1;
-      } else if (o1.firstTxId == o2.firstTxId) {
-        return 0;
-      } else {
-        return 1;
-      }
-    }
-  };
-
-  EditLogLedgerMetadata(String zkPath, int dataLayoutVersion,
-                        long ledgerId, long firstTxId) {
-    this.zkPath = zkPath;
-    this.dataLayoutVersion = dataLayoutVersion;
-    this.ledgerId = ledgerId;
-    this.firstTxId = firstTxId;
-    this.lastTxId = HdfsServerConstants.INVALID_TXID;
-    this.inprogress = true;
-  }
-  
-  EditLogLedgerMetadata(String zkPath, int dataLayoutVersion,
-                        long ledgerId, long firstTxId,
-                        long lastTxId) {
-    this.zkPath = zkPath;
-    this.dataLayoutVersion = dataLayoutVersion;
-    this.ledgerId = ledgerId;
-    this.firstTxId = firstTxId;
-    this.lastTxId = lastTxId;
-    this.inprogress = false;
-  }
-
-  String getZkPath() {
-    return zkPath;
-  }
-
-  long getFirstTxId() {
-    return firstTxId;
-  }
-  
-  long getLastTxId() {
-    return lastTxId;
-  }
-  
-  long getLedgerId() {
-    return ledgerId;
-  }
-  
-  boolean isInProgress() {
-    return this.inprogress;
-  }
-
-  int getDataLayoutVersion() {
-    return this.dataLayoutVersion;
-  }
-
-  void finalizeLedger(long newLastTxId) {
-    assert this.lastTxId == HdfsServerConstants.INVALID_TXID;
-    this.lastTxId = newLastTxId;
-    this.inprogress = false;      
-  }
-  
-  static EditLogLedgerMetadata read(ZooKeeper zkc, String path)
-      throws IOException, KeeperException.NoNodeException  {
-    try {
-      byte[] data = zkc.getData(path, false, null);
-
-      EditLogLedgerProto.Builder builder = EditLogLedgerProto.newBuilder();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Reading " + path + " data: " + new String(data, UTF_8));
-      }
-      TextFormat.merge(new String(data, UTF_8), builder);
-      if (!builder.isInitialized()) {
-        throw new IOException("Invalid/Incomplete data in znode");
-      }
-      EditLogLedgerProto ledger = builder.build();
-
-      int dataLayoutVersion = ledger.getDataLayoutVersion();
-      long ledgerId = ledger.getLedgerId();
-      long firstTxId = ledger.getFirstTxId();
-      if (ledger.hasLastTxId()) {
-        long lastTxId = ledger.getLastTxId();
-        return new EditLogLedgerMetadata(path, dataLayoutVersion,
-                                         ledgerId, firstTxId, lastTxId);
-      } else {
-        return new EditLogLedgerMetadata(path, dataLayoutVersion,
-                                         ledgerId, firstTxId);
-      }
-    } catch(KeeperException.NoNodeException nne) {
-      throw nne;
-    } catch(KeeperException ke) {
-      throw new IOException("Error reading from zookeeper", ke);
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted reading from zookeeper", ie);
-    }
-  }
-    
-  void write(ZooKeeper zkc, String path)
-      throws IOException, KeeperException.NodeExistsException {
-    this.zkPath = path;
-
-    EditLogLedgerProto.Builder builder = EditLogLedgerProto.newBuilder();
-    builder.setDataLayoutVersion(dataLayoutVersion)
-      .setLedgerId(ledgerId).setFirstTxId(firstTxId);
-
-    if (!inprogress) {
-      builder.setLastTxId(lastTxId);
-    }
-    try {
-      zkc.create(path, TextFormat.printToString(builder.build()).getBytes(UTF_8),
-                 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    } catch (KeeperException.NodeExistsException nee) {
-      throw nee;
-    } catch (KeeperException e) {
-      throw new IOException("Error creating ledger znode", e);
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted creating ledger znode", ie);
-    }
-  }
-  
-  boolean verify(ZooKeeper zkc, String path) {
-    try {
-      EditLogLedgerMetadata other = read(zkc, path);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Verifying " + this.toString() 
-                  + " against " + other);
-      }
-      return other.equals(this);
-    } catch (KeeperException e) {
-      LOG.error("Couldn't verify data in " + path, e);
-      return false;
-    } catch (IOException ie) {
-      LOG.error("Couldn't verify data in " + path, ie);
-      return false;
-    }
-  }
-  
-  public boolean equals(Object o) {
-    if (!(o instanceof EditLogLedgerMetadata)) {
-      return false;
-    }
-    EditLogLedgerMetadata ol = (EditLogLedgerMetadata)o;
-    return ledgerId == ol.ledgerId
-      && dataLayoutVersion == ol.dataLayoutVersion
-      && firstTxId == ol.firstTxId
-      && lastTxId == ol.lastTxId;
-  }
-
-  public int hashCode() {
-    int hash = 1;
-    hash = hash * 31 + (int) ledgerId;
-    hash = hash * 31 + (int) firstTxId;
-    hash = hash * 31 + (int) lastTxId;
-    hash = hash * 31 + dataLayoutVersion;
-    return hash;
-  }
-    
-  public String toString() {
-    return "[LedgerId:"+ledgerId +
-      ", firstTxId:" + firstTxId +
-      ", lastTxId:" + lastTxId +
-      ", dataLayoutVersion:" + dataLayoutVersion + "]";
-  }
-
-}

+ 0 - 103
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/MaxTxId.java

@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.data.Stat;
-
-import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.MaxTxIdProto;
-import com.google.protobuf.TextFormat;
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Utility class for storing and reading
- * the max seen txid in zookeeper
- */
-class MaxTxId {
-  static final Log LOG = LogFactory.getLog(MaxTxId.class);
-  
-  private final ZooKeeper zkc;
-  private final String path;
-
-  private Stat currentStat;
-
-  MaxTxId(ZooKeeper zkc, String path) {
-    this.zkc = zkc;
-    this.path = path;
-  }
-
-  synchronized void store(long maxTxId) throws IOException {
-    long currentMax = get();
-    if (currentMax < maxTxId) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Setting maxTxId to " + maxTxId);
-      }
-      reset(maxTxId);
-    }
-  }
-
-  synchronized void reset(long maxTxId) throws IOException {
-    try {
-      MaxTxIdProto.Builder builder = MaxTxIdProto.newBuilder().setTxId(maxTxId);
-
-      byte[] data = TextFormat.printToString(builder.build()).getBytes(UTF_8);
-      if (currentStat != null) {
-        currentStat = zkc.setData(path, data, currentStat
-            .getVersion());
-      } else {
-        zkc.create(path, data, Ids.OPEN_ACL_UNSAFE,
-                   CreateMode.PERSISTENT);
-      }
-    } catch (KeeperException e) {
-      throw new IOException("Error writing max tx id", e);
-    } catch (InterruptedException e) {
-      throw new IOException("Interrupted while writing max tx id", e);
-    }
-  }
-
-  synchronized long get() throws IOException {
-    try {
-      currentStat = zkc.exists(path, false);
-      if (currentStat == null) {
-        return 0;
-      } else {
-
-        byte[] bytes = zkc.getData(path, false, currentStat);
-
-        MaxTxIdProto.Builder builder = MaxTxIdProto.newBuilder();
-        TextFormat.merge(new String(bytes, UTF_8), builder);
-        if (!builder.isInitialized()) {
-          throw new IOException("Invalid/Incomplete data in znode");
-        }
-
-        return builder.build().getTxId();
-      }
-    } catch (KeeperException e) {
-      throw new IOException("Error reading the max tx id from zk", e);
-    } catch (InterruptedException ie) {
-      throw new IOException("Interrupted while reading thr max tx id", ie);
-    }
-  }
-}

+ 0 - 184
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/BKJMUtil.java

@@ -1,184 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import static org.junit.Assert.*;
-
-import java.net.URI;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.KeeperException;
-
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.util.LocalBookKeeper;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.List;
-
-import java.io.IOException;
-import java.io.File;
-
-/**
- * Utility class for setting up bookkeeper ensembles
- * and bringing individual bookies up and down
- */
-class BKJMUtil {
-  protected static final Log LOG = LogFactory.getLog(BKJMUtil.class);
-
-  int nextPort = 6000; // next port for additionally created bookies
-  private Thread bkthread = null;
-  private final static String zkEnsemble = "127.0.0.1:2181";
-  int numBookies;
-
-  BKJMUtil(final int numBookies) throws Exception {
-    this.numBookies = numBookies;
-
-    bkthread = new Thread() {
-        public void run() {
-          try {
-            String[] args = new String[1];
-            args[0] = String.valueOf(numBookies);
-            LOG.info("Starting bk");
-            LocalBookKeeper.main(args);
-          } catch (InterruptedException e) {
-            // go away quietly
-          } catch (Exception e) {
-            LOG.error("Error starting local bk", e);
-          }
-        }
-      };
-  }
-
-  void start() throws Exception {
-    bkthread.start();
-    if (!LocalBookKeeper.waitForServerUp(zkEnsemble, 10000)) {
-      throw new Exception("Error starting zookeeper/bookkeeper");
-    }
-    assertEquals("Not all bookies started",
-                 numBookies, checkBookiesUp(numBookies, 10));
-  }
-
-  void teardown() throws Exception {
-    if (bkthread != null) {
-      bkthread.interrupt();
-      bkthread.join();
-    }
-  }
-
-  static ZooKeeper connectZooKeeper()
-      throws IOException, KeeperException, InterruptedException {
-    final CountDownLatch latch = new CountDownLatch(1);
-
-    ZooKeeper zkc = new ZooKeeper(zkEnsemble, 3600, new Watcher() {
-        public void process(WatchedEvent event) {
-          if (event.getState() == Watcher.Event.KeeperState.SyncConnected) {
-            latch.countDown();
-          }
-        }
-      });
-    if (!latch.await(3, TimeUnit.SECONDS)) {
-      throw new IOException("Zookeeper took too long to connect");
-    }
-    return zkc;
-  }
-
-  static URI createJournalURI(String path) throws Exception {
-    return URI.create("bookkeeper://" + zkEnsemble + path);
-  }
-
-  static void addJournalManagerDefinition(Configuration conf) {
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX + ".bookkeeper",
-             "org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager");
-  }
-
-  BookieServer newBookie() throws Exception {
-    int port = nextPort++;
-    ServerConfiguration bookieConf = new ServerConfiguration();
-    bookieConf.setBookiePort(port);
-    File tmpdir = File.createTempFile("bookie" + Integer.toString(port) + "_",
-                                      "test");
-    tmpdir.delete();
-    tmpdir.mkdir();
-
-    bookieConf.setZkServers(zkEnsemble);
-    bookieConf.setJournalDirName(tmpdir.getPath());
-    bookieConf.setLedgerDirNames(new String[] { tmpdir.getPath() });
-
-    BookieServer b = new BookieServer(bookieConf);
-    b.start();
-    for (int i = 0; i < 10 && !b.isRunning(); i++) {
-      Thread.sleep(10000);
-    }
-    if (!b.isRunning()) {
-      throw new IOException("Bookie would not start");
-    }
-    return b;
-  }
-
-  /**
-   * Check that a number of bookies are available
-   * @param count number of bookies required
-   * @param timeout number of seconds to wait for bookies to start
-   * @throws IOException if bookies are not started by the time the timeout hits
-   */
-  int checkBookiesUp(int count, int timeout) throws Exception {
-    ZooKeeper zkc = connectZooKeeper();
-    try {
-      int mostRecentSize = 0;
-      for (int i = 0; i < timeout; i++) {
-        try {
-          List<String> children = zkc.getChildren("/ledgers/available",
-                                                  false);
-          mostRecentSize = children.size();
-          // Skip 'readonly znode' which is used for keeping R-O bookie details
-          if (children.contains("readonly")) {
-            mostRecentSize = children.size() - 1;
-          }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Found " + mostRecentSize + " bookies up, "
-                      + "waiting for " + count);
-            if (LOG.isTraceEnabled()) {
-              for (String child : children) {
-                LOG.trace(" server: " + child);
-              }
-            }
-          }
-          if (mostRecentSize == count) {
-            break;
-          }
-        } catch (KeeperException e) {
-          // ignore
-        }
-        Thread.sleep(1000);
-      }
-      return mostRecentSize;
-    } finally {
-      zkc.close();
-    }
-  }
-}

+ 0 - 414
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java

@@ -1,414 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ha.ServiceFailedException;
-import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
-import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-
-import org.apache.hadoop.hdfs.HAUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-
-import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-
-import org.apache.hadoop.ipc.RemoteException;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.ExitUtil;
-import org.apache.hadoop.util.ExitUtil.ExitException;
-
-import org.apache.bookkeeper.proto.BookieServer;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Collection;
-
-/**
- * Integration test to ensure that the BookKeeper JournalManager
- * works for HDFS Namenode HA
- */
-@RunWith(Parameterized.class)
-public class TestBookKeeperAsHASharedDir {
-  static final Log LOG = LogFactory.getLog(TestBookKeeperAsHASharedDir.class);
-
-  private static BKJMUtil bkutil;
-  static int numBookies = 3;
-
-  private static final String TEST_FILE_DATA = "HA BookKeeperJournalManager";
-
-  @Parameters
-  public static Collection<Object[]> data() {
-    Collection<Object[]> params = new ArrayList<Object[]>();
-    params.add(new Object[]{ Boolean.FALSE });
-    params.add(new Object[]{ Boolean.TRUE });
-    return params;
-  }
-
-  private static boolean useAsyncEditLog;
-  public TestBookKeeperAsHASharedDir(Boolean async) {
-    useAsyncEditLog = async;
-  }
-
-  private static Configuration getConf() {
-    Configuration conf = new Configuration();
-    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_ASYNC_LOGGING,
-        useAsyncEditLog);
-    return conf;
-  }
-
-  @BeforeClass
-  public static void setupBookkeeper() throws Exception {
-    bkutil = new BKJMUtil(numBookies);
-    bkutil.start();
-  }
-  
-  @Before
-  public void clearExitStatus() {
-    ExitUtil.resetFirstExitException();
-  }
-
-  @AfterClass
-  public static void teardownBookkeeper() throws Exception {
-    bkutil.teardown();
-  }
-
-  /**
-   * Test simple HA failover usecase with BK
-   */
-  @Test
-  public void testFailoverWithBK() throws Exception {
-    MiniDFSCluster cluster = null;
-    try {
-      Configuration conf = getConf();
-      conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-               BKJMUtil.createJournalURI("/hotfailover").toString());
-      BKJMUtil.addJournalManagerDefinition(conf);
-
-      cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
-        .numDataNodes(0)
-        .manageNameDfsSharedDirs(false)
-        .build();
-      NameNode nn1 = cluster.getNameNode(0);
-      NameNode nn2 = cluster.getNameNode(1);
-
-      cluster.waitActive();
-      cluster.transitionToActive(0);
-
-      Path p = new Path("/testBKJMfailover");
-
-      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
-
-      fs.mkdirs(p);
-      cluster.shutdownNameNode(0);
-
-      cluster.transitionToActive(1);
-
-      assertTrue(fs.exists(p));
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  /**
-   * Test HA failover, where BK, as the shared storage, fails.
-   * Once it becomes available again, a standby can come up.
-   * Verify that any write happening after the BK fail is not
-   * available on the standby.
-   */
-  @Test
-  public void testFailoverWithFailingBKCluster() throws Exception {
-    int ensembleSize = numBookies + 1;
-    BookieServer newBookie = bkutil.newBookie();
-    assertEquals("New bookie didn't start",
-                 ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-    BookieServer replacementBookie = null;
-
-    MiniDFSCluster cluster = null;
-
-    try {
-      Configuration conf = getConf();
-      conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-               BKJMUtil.createJournalURI("/hotfailoverWithFail").toString());
-      conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-                  ensembleSize);
-      conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
-                  ensembleSize);
-      BKJMUtil.addJournalManagerDefinition(conf);
-
-      cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
-        .numDataNodes(0)
-        .manageNameDfsSharedDirs(false)
-        .checkExitOnShutdown(false)
-        .build();
-      NameNode nn1 = cluster.getNameNode(0);
-      NameNode nn2 = cluster.getNameNode(1);
-
-      cluster.waitActive();
-      cluster.transitionToActive(0);
-
-      Path p1 = new Path("/testBKJMFailingBKCluster1");
-      Path p2 = new Path("/testBKJMFailingBKCluster2");
-
-      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
-
-      fs.mkdirs(p1);
-      newBookie.shutdown(); // will take down shared storage
-      assertEquals("New bookie didn't stop",
-                   numBookies, bkutil.checkBookiesUp(numBookies, 10));
-
-      try {
-        fs.mkdirs(p2);
-        fail("mkdirs should result in the NN exiting");
-      } catch (RemoteException re) {
-        assertTrue(re.getClassName().contains("ExitException"));
-      }
-      cluster.shutdownNameNode(0);
-
-      try {
-        cluster.transitionToActive(1);
-        fail("Shouldn't have been able to transition with bookies down");
-      } catch (ExitException ee) {
-        assertTrue("Should shutdown due to required journal failure",
-            ee.getMessage().contains(
-                "starting log segment 3 failed for required journal"));
-      }
-
-      replacementBookie = bkutil.newBookie();
-      assertEquals("Replacement bookie didn't start",
-                   ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-      cluster.transitionToActive(1); // should work fine now
-
-      assertTrue(fs.exists(p1));
-      assertFalse(fs.exists(p2));
-    } finally {
-      newBookie.shutdown();
-      if (replacementBookie != null) {
-        replacementBookie.shutdown();
-      }
-
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  /**
-   * Test that two namenodes can't continue as primary
-   */
-  @Test
-  public void testMultiplePrimariesStarted() throws Exception {
-    Path p1 = new Path("/testBKJMMultiplePrimary");
-
-    MiniDFSCluster cluster = null;
-    try {
-      Configuration conf = getConf();
-      conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-               BKJMUtil.createJournalURI("/hotfailoverMultiple").toString());
-      BKJMUtil.addJournalManagerDefinition(conf);
-
-      cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
-        .numDataNodes(0)
-        .manageNameDfsSharedDirs(false)
-        .checkExitOnShutdown(false)
-        .build();
-      NameNode nn1 = cluster.getNameNode(0);
-      NameNode nn2 = cluster.getNameNode(1);
-      cluster.waitActive();
-      cluster.transitionToActive(0);
-
-      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
-      fs.mkdirs(p1);
-      nn1.getRpcServer().rollEditLog();
-      cluster.transitionToActive(1);
-      fs = cluster.getFileSystem(0); // get the older active server.
-
-      try {
-        System.out.println("DMS: > *************");
-        boolean foo = fs.delete(p1, true);
-        System.out.println("DMS: < ************* "+foo);
-        fail("Log update on older active should cause it to exit");
-      } catch (RemoteException re) {
-        assertTrue(re.getClassName().contains("ExitException"));
-      }
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-  
-  /**
-   * Use NameNode INTIALIZESHAREDEDITS to initialize the shared edits. i.e. copy
-   * the edits log segments to new bkjm shared edits.
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testInitializeBKSharedEdits() throws Exception {
-    MiniDFSCluster cluster = null;
-    try {
-      Configuration conf = getConf();
-      HAUtil.setAllowStandbyReads(conf, true);
-
-      MiniDFSNNTopology topology = MiniDFSNNTopology.simpleHATopology();
-      cluster = new MiniDFSCluster.Builder(conf).nnTopology(topology)
-          .numDataNodes(0).build();
-      cluster.waitActive();
-      // Shutdown and clear the current filebased shared dir.
-      cluster.shutdownNameNodes();
-      File shareddir = new File(cluster.getSharedEditsDir(0, 1));
-      assertTrue("Initial Shared edits dir not fully deleted",
-          FileUtil.fullyDelete(shareddir));
-
-      // Check namenodes should not start without shared dir.
-      assertCanNotStartNamenode(cluster, 0);
-      assertCanNotStartNamenode(cluster, 1);
-
-      // Configure bkjm as new shared edits dir in both namenodes
-      Configuration nn1Conf = cluster.getConfiguration(0);
-      Configuration nn2Conf = cluster.getConfiguration(1);
-      nn1Conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
-          .createJournalURI("/initializeSharedEdits").toString());
-      nn2Conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
-          .createJournalURI("/initializeSharedEdits").toString());
-      BKJMUtil.addJournalManagerDefinition(nn1Conf);
-      BKJMUtil.addJournalManagerDefinition(nn2Conf);
-
-      // Initialize the BKJM shared edits.
-      assertFalse(NameNode.initializeSharedEdits(nn1Conf));
-
-      // NameNode should be able to start and should be in sync with BKJM as
-      // shared dir
-      assertCanStartHANameNodes(cluster, conf, "/testBKJMInitialize");
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  private void assertCanNotStartNamenode(MiniDFSCluster cluster, int nnIndex) {
-    try {
-      cluster.restartNameNode(nnIndex, false);
-      fail("Should not have been able to start NN" + (nnIndex)
-          + " without shared dir");
-    } catch (IOException ioe) {
-      LOG.info("Got expected exception", ioe);
-      GenericTestUtils.assertExceptionContains(
-          "storage directory does not exist or is not accessible", ioe);
-    }
-  }
-
-  private void assertCanStartHANameNodes(MiniDFSCluster cluster,
-      Configuration conf, String path) throws ServiceFailedException,
-      IOException, URISyntaxException, InterruptedException {
-    // Now should be able to start both NNs. Pass "false" here so that we don't
-    // try to waitActive on all NNs, since the second NN doesn't exist yet.
-    cluster.restartNameNode(0, false);
-    cluster.restartNameNode(1, true);
-
-    // Make sure HA is working.
-    cluster
-        .getNameNode(0)
-        .getRpcServer()
-        .transitionToActive(
-            new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER));
-    FileSystem fs = null;
-    try {
-      Path newPath = new Path(path);
-      fs = HATestUtil.configureFailoverFs(cluster, conf);
-      assertTrue(fs.mkdirs(newPath));
-      HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
-          cluster.getNameNode(1));
-      assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-          newPath.toString(), false).isDir());
-    } finally {
-      if (fs != null) {
-        fs.close();
-      }
-    }
-  }
-
-  /**
-   * NameNode should load the edits correctly if the applicable edits are
-   * present in the BKJM.
-   */
-  @Test
-  public void testNameNodeMultipleSwitchesUsingBKJM() throws Exception {
-    MiniDFSCluster cluster = null;
-    try {
-      Configuration conf = getConf();
-      conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
-          .createJournalURI("/correctEditLogSelection").toString());
-      BKJMUtil.addJournalManagerDefinition(conf);
-
-      cluster = new MiniDFSCluster.Builder(conf)
-          .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
-          .manageNameDfsSharedDirs(false).build();
-      NameNode nn1 = cluster.getNameNode(0);
-      NameNode nn2 = cluster.getNameNode(1);
-      cluster.waitActive();
-      cluster.transitionToActive(0);
-      nn1.getRpcServer().rollEditLog(); // Roll Edits from current Active.
-      // Transition to standby current active gracefully.
-      cluster.transitionToStandby(0);
-      // Make the other Active and Roll edits multiple times
-      cluster.transitionToActive(1);
-      nn2.getRpcServer().rollEditLog();
-      nn2.getRpcServer().rollEditLog();
-      // Now One more failover. So NN1 should be able to failover successfully.
-      cluster.transitionToStandby(1);
-      cluster.transitionToActive(0);
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-}

+ 0 - 174
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperConfiguration.java

@@ -1,174 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.Random;
-
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-
-public class TestBookKeeperConfiguration {
-  private static final Log LOG = LogFactory
-      .getLog(TestBookKeeperConfiguration.class);
-  private static final int ZK_SESSION_TIMEOUT = 5000;
-  private static final String HOSTPORT = "127.0.0.1:2181";
-  private static final int CONNECTION_TIMEOUT = 30000;
-  private static NIOServerCnxnFactory serverFactory;
-  private static ZooKeeperServer zks;
-  private static ZooKeeper zkc;
-  private static int ZooKeeperDefaultPort = 2181;
-  private static File ZkTmpDir;
-  private BookKeeperJournalManager bkjm;
-  private static final String BK_ROOT_PATH = "/ledgers";
-
-  private static ZooKeeper connectZooKeeper(String ensemble)
-      throws IOException, KeeperException, InterruptedException {
-    final CountDownLatch latch = new CountDownLatch(1);
-
-    ZooKeeper zkc = new ZooKeeper(HOSTPORT, ZK_SESSION_TIMEOUT, new Watcher() {
-      public void process(WatchedEvent event) {
-        if (event.getState() == Watcher.Event.KeeperState.SyncConnected) {
-          latch.countDown();
-        }
-      }
-    });
-    if (!latch.await(ZK_SESSION_TIMEOUT, TimeUnit.MILLISECONDS)) {
-      throw new IOException("Zookeeper took too long to connect");
-    }
-    return zkc;
-  }
-
-  private NamespaceInfo newNSInfo() {
-    Random r = new Random();
-    return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
-  }
-
-  @BeforeClass
-  public static void setupZooKeeper() throws Exception {
-    // create a ZooKeeper server(dataDir, dataLogDir, port)
-    LOG.info("Starting ZK server");
-    ZkTmpDir = File.createTempFile("zookeeper", "test");
-    ZkTmpDir.delete();
-    ZkTmpDir.mkdir();
-
-    try {
-      zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
-      serverFactory = new NIOServerCnxnFactory();
-      serverFactory.configure(new InetSocketAddress(ZooKeeperDefaultPort), 10);
-      serverFactory.startup(zks);
-    } catch (Exception e) {
-      LOG.error("Exception while instantiating ZooKeeper", e);
-    }
-
-    boolean b = LocalBookKeeper.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT);
-    LOG.debug("ZooKeeper server up: " + b);
-  }
-
-  @Before
-  public void setup() throws Exception {
-    zkc = connectZooKeeper(HOSTPORT);
-    try {
-      ZKUtil.deleteRecursive(zkc, BK_ROOT_PATH);
-    } catch (KeeperException.NoNodeException e) {
-      LOG.debug("Ignoring no node exception on cleanup", e);
-    } catch (Exception e) {
-      LOG.error("Exception when deleting bookie root path in zk", e);
-    }
-  }
-
-  @After
-  public void teardown() throws Exception {
-    if (null != zkc) {
-      zkc.close();
-    }
-    if (null != bkjm) {
-      bkjm.close();
-    }
-  }
-
-  @AfterClass
-  public static void teardownZooKeeper() throws Exception {
-    if (null != zkc) {
-      zkc.close();
-    }
-  }
-
-  /**
-   * Verify the BKJM is creating the bookie available path configured in
-   * 'dfs.namenode.bookkeeperjournal.zk.availablebookies'
-   */
-  @Test
-  public void testWithConfiguringBKAvailablePath() throws Exception {
-    // set Bookie available path in the configuration
-    String bkAvailablePath 
-      = BookKeeperJournalManager.BKJM_ZK_LEDGERS_AVAILABLE_PATH_DEFAULT;
-    Configuration conf = new Configuration();
-    conf.setStrings(BookKeeperJournalManager.BKJM_ZK_LEDGERS_AVAILABLE_PATH,
-        bkAvailablePath);
-    Assert.assertNull(bkAvailablePath + " already exists", zkc.exists(
-        bkAvailablePath, false));
-    NamespaceInfo nsi = newNSInfo();
-    bkjm = new BookKeeperJournalManager(conf,
-        URI.create("bookkeeper://" + HOSTPORT + "/hdfsjournal-WithBKPath"),
-        nsi);
-    bkjm.format(nsi);
-    Assert.assertNotNull("Bookie available path : " + bkAvailablePath
-        + " doesn't exists", zkc.exists(bkAvailablePath, false));
-  }
-
-  /**
-   * Verify the BKJM is creating the bookie available default path, when there
-   * is no 'dfs.namenode.bookkeeperjournal.zk.availablebookies' configured
-   */
-  @Test
-  public void testDefaultBKAvailablePath() throws Exception {
-    Configuration conf = new Configuration();
-    Assert.assertNull(BK_ROOT_PATH + " already exists", zkc.exists(
-        BK_ROOT_PATH, false));
-    NamespaceInfo nsi = newNSInfo();
-    bkjm = new BookKeeperJournalManager(conf,
-        URI.create("bookkeeper://" + HOSTPORT + "/hdfsjournal-DefaultBKPath"),
-        nsi);
-    bkjm.format(nsi);
-    Assert.assertNotNull("Bookie available path : " + BK_ROOT_PATH
-        + " doesn't exists", zkc.exists(BK_ROOT_PATH, false));
-  }
-}

+ 0 - 92
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java

@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Unit test for the bkjm's streams
- */
-public class TestBookKeeperEditLogStreams {
-  static final Log LOG = LogFactory.getLog(TestBookKeeperEditLogStreams.class);
-
-  private static BKJMUtil bkutil;
-  private final static int numBookies = 3;
-
-  @BeforeClass
-  public static void setupBookkeeper() throws Exception {
-    bkutil = new BKJMUtil(numBookies);
-    bkutil.start();
-  }
-
-  @AfterClass
-  public static void teardownBookkeeper() throws Exception {
-    bkutil.teardown();
-  }
-
-  /**
-   * Test that bkjm will refuse open a stream on an empty
-   * ledger.
-   */
-  @Test
-  public void testEmptyInputStream() throws Exception {
-    ZooKeeper zk = BKJMUtil.connectZooKeeper();
-
-    BookKeeper bkc = new BookKeeper(new ClientConfiguration(), zk);
-    try {
-      LedgerHandle lh = bkc.createLedger(BookKeeper.DigestType.CRC32, "foobar"
-          .getBytes());
-      lh.close();
-
-      EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
-      try {
-        new BookKeeperEditLogInputStream(lh, metadata, -1);
-        fail("Shouldn't get this far, should have thrown");
-      } catch (IOException ioe) {
-        assertTrue(ioe.getMessage().contains("Invalid first bk entry to read"));
-      }
-
-      metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
-      try {
-        new BookKeeperEditLogInputStream(lh, metadata, 0);
-        fail("Shouldn't get this far, should have thrown");
-      } catch (IOException ioe) {
-        assertTrue(ioe.getMessage().contains("Invalid first bk entry to read"));
-      }
-    } finally {
-      bkc.close();
-      zk.close();
-    }
-  }
-}

+ 0 - 109
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java

@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-import java.net.BindException;
-import java.util.Random;
-
-/**
- * Runs the same tests as TestStandbyCheckpoints, but
- * using a bookkeeper journal manager as the shared directory
- */
-public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
-  //overwrite the nn count
- static{
-   TestStandbyCheckpoints.NUM_NNS = 2;
- }
-  private static BKJMUtil bkutil = null;
-  static int numBookies = 3;
-  static int journalCount = 0;
-  private final Random random = new Random();
-
-  private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
-
-  @SuppressWarnings("rawtypes")
-  @Override
-  @Before
-  public void setupCluster() throws Exception {
-    Configuration conf = setupCommonConfig();
-    conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-             BKJMUtil.createJournalURI("/checkpointing" + journalCount++)
-             .toString());
-    BKJMUtil.addJournalManagerDefinition(conf);
-
-    int retryCount = 0;
-    while (true) {
-      try {
-        int basePort = 10060 + random.nextInt(100) * 2;
-        MiniDFSNNTopology topology = new MiniDFSNNTopology()
-          .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-            .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
-            .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
-
-        cluster = new MiniDFSCluster.Builder(conf)
-          .nnTopology(topology)
-          .numDataNodes(1)
-          .manageNameDfsSharedDirs(false)
-          .build();
-        cluster.waitActive();
-
-        setNNs();
-        fs = HATestUtil.configureFailoverFs(cluster, conf);
-
-        cluster.transitionToActive(0);
-        ++retryCount;
-        break;
-      } catch (BindException e) {
-        LOG.info("Set up MiniDFSCluster failed due to port conflicts, retry "
-            + retryCount + " times");
-      }
-    }
-  }
-
-  @BeforeClass
-  public static void startBK() throws Exception {
-    journalCount = 0;
-    bkutil = new BKJMUtil(numBookies);
-    bkutil.start();
-  }
-
-  @AfterClass
-  public static void shutdownBK() throws Exception {
-    if (bkutil != null) {
-      bkutil.teardown();
-    }
-  }
-
-  @Override
-  public void testCheckpointCancellation() throws Exception {
-    // Overriden as the implementation in the superclass assumes that writes
-    // are to a file. This should be fixed at some point
-  }
-}

+ 0 - 984
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java

@@ -1,984 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.spy;
-import org.junit.Test;
-import org.junit.Before;
-import org.junit.After;
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.mockito.Mockito;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-
-import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
-import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
-import org.apache.hadoop.hdfs.server.namenode.JournalManager;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooDefs.Ids;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-public class TestBookKeeperJournalManager {
-  static final Log LOG = LogFactory.getLog(TestBookKeeperJournalManager.class);
-  
-  private static final long DEFAULT_SEGMENT_SIZE = 1000;
-
-  protected static Configuration conf = new Configuration();
-  private ZooKeeper zkc;
-  private static BKJMUtil bkutil;
-  static int numBookies = 3;
-  private BookieServer newBookie;
-
-  @BeforeClass
-  public static void setupBookkeeper() throws Exception {
-    bkutil = new BKJMUtil(numBookies);
-    bkutil.start();
-  }
-
-  @AfterClass
-  public static void teardownBookkeeper() throws Exception {
-    bkutil.teardown();
-  }
-
-  @Before
-  public void setup() throws Exception {
-    zkc = BKJMUtil.connectZooKeeper();
-  }
-
-  @After
-  public void teardown() throws Exception {
-    zkc.close();
-    if (newBookie != null) {
-      newBookie.shutdown();
-    }
-  }
-
-  private NamespaceInfo newNSInfo() {
-    Random r = new Random();
-    return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
-  }
-
-  @Test
-  public void testSimpleWrite() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-simplewrite"), nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long i = 1 ; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, 100);
- 
-    String zkpath = bkjm.finalizedLedgerZNode(1, 100);
-    
-    assertNotNull(zkc.exists(zkpath, false));
-    assertNull(zkc.exists(bkjm.inprogressZNode(1), false));
-  }
-
-  @Test
-  public void testNumberOfTransactions() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-txncount"), nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long i = 1 ; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, 100);
-
-    long numTrans = bkjm.getNumberOfTransactions(1, true);
-    assertEquals(100, numTrans);
-  }
-
-  @Test 
-  public void testNumberOfTransactionsWithGaps() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-gaps"), nsi);
-    bkjm.format(nsi);
-
-    long txid = 1;
-    for (long i = 0; i < 3; i++) {
-      long start = txid;
-      EditLogOutputStream out = bkjm.startLogSegment(start,
-          NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
-        FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-        op.setTransactionId(txid++);
-        out.write(op);
-      }
-      out.close();
-      bkjm.finalizeLogSegment(start, txid-1);
-      assertNotNull(
-          zkc.exists(bkjm.finalizedLedgerZNode(start, txid-1), false));
-    }
-    zkc.delete(bkjm.finalizedLedgerZNode(DEFAULT_SEGMENT_SIZE+1,
-                                         DEFAULT_SEGMENT_SIZE*2), -1);
-    
-    long numTrans = bkjm.getNumberOfTransactions(1, true);
-    assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
-    
-    try {
-      numTrans = bkjm.getNumberOfTransactions(DEFAULT_SEGMENT_SIZE+1, true);
-      fail("Should have thrown corruption exception by this point");
-    } catch (JournalManager.CorruptionException ce) {
-      // if we get here, everything is going good
-    }
-
-    numTrans = bkjm.getNumberOfTransactions((DEFAULT_SEGMENT_SIZE*2)+1, true);
-    assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
-  }
-
-  @Test
-  public void testNumberOfTransactionsWithInprogressAtEnd() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-inprogressAtEnd"), nsi);
-    bkjm.format(nsi);
-
-    long txid = 1;
-    for (long i = 0; i < 3; i++) {
-      long start = txid;
-      EditLogOutputStream out = bkjm.startLogSegment(start,
-          NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
-        FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-        op.setTransactionId(txid++);
-        out.write(op);
-      }
-      
-      out.close();
-      bkjm.finalizeLogSegment(start, (txid-1));
-      assertNotNull(
-          zkc.exists(bkjm.finalizedLedgerZNode(start, (txid-1)), false));
-    }
-    long start = txid;
-    EditLogOutputStream out = bkjm.startLogSegment(start,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE/2; j++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(txid++);
-      out.write(op);
-    }
-    out.setReadyToFlush();
-    out.flush();
-    out.abort();
-    out.close();
-    
-    long numTrans = bkjm.getNumberOfTransactions(1, true);
-    assertEquals((txid-1), numTrans);
-  }
-
-  /**
-   * Create a bkjm namespace, write a journal from txid 1, close stream.
-   * Try to create a new journal from txid 1. Should throw an exception.
-   */
-  @Test
-  public void testWriteRestartFrom1() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-restartFrom1"), nsi);
-    bkjm.format(nsi);
-
-    long txid = 1;
-    long start = txid;
-    EditLogOutputStream out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(txid++);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(start, (txid-1));
-    
-    txid = 1;
-    try {
-      out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      fail("Shouldn't be able to start another journal from " + txid
-          + " when one already exists");
-    } catch (Exception ioe) {
-      LOG.info("Caught exception as expected", ioe);
-    }
-
-    // test border case
-    txid = DEFAULT_SEGMENT_SIZE;
-    try {
-      out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      fail("Shouldn't be able to start another journal from " + txid
-          + " when one already exists");
-    } catch (IOException ioe) {
-      LOG.info("Caught exception as expected", ioe);
-    }
-
-    // open journal continuing from before
-    txid = DEFAULT_SEGMENT_SIZE + 1;
-    start = txid;
-    out = bkjm.startLogSegment(start,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    assertNotNull(out);
-
-    for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(txid++);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(start, (txid-1));
-
-    // open journal arbitarily far in the future
-    txid = DEFAULT_SEGMENT_SIZE * 4;
-    out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    assertNotNull(out);
-  }
-
-  @Test
-  public void testTwoWriters() throws Exception {
-    long start = 1;
-    NamespaceInfo nsi = newNSInfo();
-
-    BookKeeperJournalManager bkjm1 = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"), nsi);
-    bkjm1.format(nsi);
-
-    BookKeeperJournalManager bkjm2 = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"), nsi);
-
-
-    EditLogOutputStream out1 = bkjm1.startLogSegment(start,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    try {
-      bkjm2.startLogSegment(start,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      fail("Shouldn't have been able to open the second writer");
-    } catch (IOException ioe) {
-      LOG.info("Caught exception as expected", ioe);
-    }finally{
-      out1.close();
-    }
-  }
-
-  @Test
-  public void testSimpleRead() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-simpleread"),
-        nsi);
-    bkjm.format(nsi);
-
-    final long numTransactions = 10000;
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
-    for (long i = 1 ; i <= numTransactions; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, numTransactions);
-
-    List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
-    bkjm.selectInputStreams(in, 1, true);
-    try {
-      assertEquals(numTransactions, 
-                   FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
-    } finally {
-      in.get(0).close();
-    }
-  }
-
-  @Test
-  public void testSimpleRecovery() throws Exception {
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-simplerecovery"),
-        nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
-    for (long i = 1 ; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.setReadyToFlush();
-    out.flush();
-
-    out.abort();
-    out.close();
-
-
-    assertNull(zkc.exists(bkjm.finalizedLedgerZNode(1, 100), false));
-    assertNotNull(zkc.exists(bkjm.inprogressZNode(1), false));
-
-    bkjm.recoverUnfinalizedSegments();
-
-    assertNotNull(zkc.exists(bkjm.finalizedLedgerZNode(1, 100), false));
-    assertNull(zkc.exists(bkjm.inprogressZNode(1), false));
-  }
-
-  /**
-   * Test that if enough bookies fail to prevent an ensemble,
-   * writes the bookkeeper will fail. Test that when once again
-   * an ensemble is available, it can continue to write.
-   */
-  @Test
-  public void testAllBookieFailure() throws Exception {
-    // bookie to fail
-    newBookie = bkutil.newBookie();
-    BookieServer replacementBookie = null;
-
-    try {
-      int ensembleSize = numBookies + 1;
-      assertEquals("New bookie didn't start",
-                   ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-      // ensure that the journal manager has to use all bookies,
-      // so that a failure will fail the journal manager
-      Configuration conf = new Configuration();
-      conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-                  ensembleSize);
-      conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
-                  ensembleSize);
-      long txid = 1;
-      NamespaceInfo nsi = newNSInfo();
-      BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-          BKJMUtil.createJournalURI("/hdfsjournal-allbookiefailure"),
-          nsi);
-      bkjm.format(nsi);
-      EditLogOutputStream out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-
-      for (long i = 1 ; i <= 3; i++) {
-        FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-        op.setTransactionId(txid++);
-        out.write(op);
-      }
-      out.setReadyToFlush();
-      out.flush();
-      newBookie.shutdown();
-      assertEquals("New bookie didn't die",
-                   numBookies, bkutil.checkBookiesUp(numBookies, 10));
-
-      try {
-        for (long i = 1 ; i <= 3; i++) {
-          FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-          op.setTransactionId(txid++);
-          out.write(op);
-        }
-        out.setReadyToFlush();
-        out.flush();
-        fail("should not get to this stage");
-      } catch (IOException ioe) {
-        LOG.debug("Error writing to bookkeeper", ioe);
-        assertTrue("Invalid exception message",
-                   ioe.getMessage().contains("Failed to write to bookkeeper"));
-      }
-      replacementBookie = bkutil.newBookie();
-
-      assertEquals("New bookie didn't start",
-                   numBookies+1, bkutil.checkBookiesUp(numBookies+1, 10));
-      bkjm.recoverUnfinalizedSegments();
-      out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      for (long i = 1 ; i <= 3; i++) {
-        FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-        op.setTransactionId(txid++);
-        out.write(op);
-      }
-
-      out.setReadyToFlush();
-      out.flush();
-
-    } catch (Exception e) {
-      LOG.error("Exception in test", e);
-      throw e;
-    } finally {
-      if (replacementBookie != null) {
-        replacementBookie.shutdown();
-      }
-      newBookie.shutdown();
-
-      if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
-        LOG.warn("Not all bookies from this test shut down, expect errors");
-      }
-    }
-  }
-
-  /**
-   * Test that a BookKeeper JM can continue to work across the
-   * failure of a bookie. This should be handled transparently
-   * by bookkeeper.
-   */
-  @Test
-  public void testOneBookieFailure() throws Exception {
-    newBookie = bkutil.newBookie();
-    BookieServer replacementBookie = null;
-
-    try {
-      int ensembleSize = numBookies + 1;
-      assertEquals("New bookie didn't start",
-                   ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-      // ensure that the journal manager has to use all bookies,
-      // so that a failure will fail the journal manager
-      Configuration conf = new Configuration();
-      conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-                  ensembleSize);
-      conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
-                  ensembleSize);
-      long txid = 1;
-
-      NamespaceInfo nsi = newNSInfo();
-      BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-          BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"),
-          nsi);
-      bkjm.format(nsi);
-
-      EditLogOutputStream out = bkjm.startLogSegment(txid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      for (long i = 1 ; i <= 3; i++) {
-        FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-        op.setTransactionId(txid++);
-        out.write(op);
-      }
-      out.setReadyToFlush();
-      out.flush();
-
-      replacementBookie = bkutil.newBookie();
-      assertEquals("replacement bookie didn't start",
-                   ensembleSize+1, bkutil.checkBookiesUp(ensembleSize+1, 10));
-      newBookie.shutdown();
-      assertEquals("New bookie didn't die",
-                   ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
-      for (long i = 1 ; i <= 3; i++) {
-        FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-        op.setTransactionId(txid++);
-        out.write(op);
-      }
-      out.setReadyToFlush();
-      out.flush();
-    } catch (Exception e) {
-      LOG.error("Exception in test", e);
-      throw e;
-    } finally {
-      if (replacementBookie != null) {
-        replacementBookie.shutdown();
-      }
-      newBookie.shutdown();
-
-      if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
-        LOG.warn("Not all bookies from this test shut down, expect errors");
-      }
-    }
-  }
-  
-  /**
-   * If a journal manager has an empty inprogress node, ensure that we throw an
-   * error, as this should not be possible, and some third party has corrupted
-   * the zookeeper state
-   */
-  @Test
-  public void testEmptyInprogressNode() throws Exception {
-    URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogress");
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
-                                                                 nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
-    for (long i = 1; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, 100);
-
-    out = bkjm.startLogSegment(101,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    out.close();
-    bkjm.close();
-    String inprogressZNode = bkjm.inprogressZNode(101);
-    zkc.setData(inprogressZNode, new byte[0], -1);
-
-    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
-    try {
-      bkjm.recoverUnfinalizedSegments();
-      fail("Should have failed. There should be no way of creating"
-          + " an empty inprogess znode");
-    } catch (IOException e) {
-      // correct behaviour
-      assertTrue("Exception different than expected", e.getMessage().contains(
-          "Invalid/Incomplete data in znode"));
-    } finally {
-      bkjm.close();
-    }
-  }
-
-  /**
-   * If a journal manager has an corrupt inprogress node, ensure that we throw
-   * an error, as this should not be possible, and some third party has
-   * corrupted the zookeeper state
-   */
-  @Test
-  public void testCorruptInprogressNode() throws Exception {
-    URI uri = BKJMUtil.createJournalURI("/hdfsjournal-corruptInprogress");
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
-                                                                 nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
-    for (long i = 1; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, 100);
-
-    out = bkjm.startLogSegment(101,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    out.close();
-    bkjm.close();
-
-    String inprogressZNode = bkjm.inprogressZNode(101);
-    zkc.setData(inprogressZNode, "WholeLottaJunk".getBytes(), -1);
-
-    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
-    try {
-      bkjm.recoverUnfinalizedSegments();
-      fail("Should have failed. There should be no way of creating"
-          + " an empty inprogess znode");
-    } catch (IOException e) {
-      // correct behaviour
-      assertTrue("Exception different than expected", e.getMessage().contains(
-          "has no field named"));
-    } finally {
-      bkjm.close();
-    }
-  }
-
-  /**
-   * Cases can occur where we create a segment but crash before we even have the
-   * chance to write the START_SEGMENT op. If this occurs we should warn, but
-   * load as normal
-   */
-  @Test
-  public void testEmptyInprogressLedger() throws Exception {
-    URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogressLedger");
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
-                                                                 nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
-    for (long i = 1; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, 100);
-
-    out = bkjm.startLogSegment(101,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    out.close();
-    bkjm.close();
-
-    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
-    bkjm.recoverUnfinalizedSegments();
-    out = bkjm.startLogSegment(101,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long i = 1; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(101, 200);
-
-    bkjm.close();
-  }
-
-  /**
-   * Test that if we fail between finalizing an inprogress and deleting the
-   * corresponding inprogress znode.
-   */
-  @Test
-  public void testRefinalizeAlreadyFinalizedInprogress() throws Exception {
-    URI uri = BKJMUtil
-        .createJournalURI("/hdfsjournal-refinalizeInprogressLedger");
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
-                                                                 nsi);
-    bkjm.format(nsi);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
-    for (long i = 1; i <= 100; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.close();
-
-    String inprogressZNode = bkjm.inprogressZNode(1);
-    String finalizedZNode = bkjm.finalizedLedgerZNode(1, 100);
-    assertNotNull("inprogress znode doesn't exist", zkc.exists(inprogressZNode,
-        null));
-    assertNull("finalized znode exists", zkc.exists(finalizedZNode, null));
-
-    byte[] inprogressData = zkc.getData(inprogressZNode, false, null);
-
-    // finalize
-    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
-    bkjm.recoverUnfinalizedSegments();
-    bkjm.close();
-
-    assertNull("inprogress znode exists", zkc.exists(inprogressZNode, null));
-    assertNotNull("finalized znode doesn't exist", zkc.exists(finalizedZNode,
-        null));
-
-    zkc.create(inprogressZNode, inprogressData, Ids.OPEN_ACL_UNSAFE,
-        CreateMode.PERSISTENT);
-
-    // should work fine
-    bkjm = new BookKeeperJournalManager(conf, uri, nsi);
-    bkjm.recoverUnfinalizedSegments();
-    bkjm.close();
-  }
-
-  /**
-   * Tests that the edit log file meta data reading from ZooKeeper should be
-   * able to handle the NoNodeException. bkjm.getInputStream(fromTxId,
-   * inProgressOk) should suppress the NoNodeException and continue. HDFS-3441.
-   */
-  @Test
-  public void testEditLogFileNotExistsWhenReadingMetadata() throws Exception {
-    URI uri = BKJMUtil.createJournalURI("/hdfsjournal-editlogfile");
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
-                                                                 nsi);
-    bkjm.format(nsi);
-
-    try {
-      // start new inprogress log segment with txid=1
-      // and write transactions till txid=50
-      String zkpath1 = startAndFinalizeLogSegment(bkjm, 1, 50);
-
-      // start new inprogress log segment with txid=51
-      // and write transactions till txid=100
-      String zkpath2 = startAndFinalizeLogSegment(bkjm, 51, 100);
-
-      // read the metadata from ZK. Here simulating the situation
-      // when reading,the edit log metadata can be removed by purger thread.
-      ZooKeeper zkspy = spy(BKJMUtil.connectZooKeeper());
-      bkjm.setZooKeeper(zkspy);
-      Mockito.doThrow(
-          new KeeperException.NoNodeException(zkpath2 + " doesn't exists"))
-          .when(zkspy).getData(zkpath2, false, null);
-
-      List<EditLogLedgerMetadata> ledgerList = bkjm.getLedgerList(false);
-      assertEquals("List contains the metadata of non exists path.", 1,
-          ledgerList.size());
-      assertEquals("LogLedgerMetadata contains wrong zk paths.", zkpath1,
-          ledgerList.get(0).getZkPath());
-    } finally {
-      bkjm.close();
-    }
-  }
-
-  private enum ThreadStatus {
-    COMPLETED, GOODEXCEPTION, BADEXCEPTION;
-  };
-
-  /**
-   * Tests that concurrent calls to format will still allow one to succeed.
-   */
-  @Test
-  public void testConcurrentFormat() throws Exception {
-    final URI uri = BKJMUtil.createJournalURI("/hdfsjournal-concurrentformat");
-    final NamespaceInfo nsi = newNSInfo();
-
-    // populate with data first
-    BookKeeperJournalManager bkjm
-      = new BookKeeperJournalManager(conf, uri, nsi);
-    bkjm.format(nsi);
-    for (int i = 1; i < 100*2; i += 2) {
-      bkjm.startLogSegment(i, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-      bkjm.finalizeLogSegment(i, i+1);
-    }
-    bkjm.close();
-
-    final int numThreads = 40;
-    List<Callable<ThreadStatus>> threads
-      = new ArrayList<Callable<ThreadStatus>>();
-    final CyclicBarrier barrier = new CyclicBarrier(numThreads);
-
-    for (int i = 0; i < numThreads; i++) {
-      threads.add(new Callable<ThreadStatus>() {
-          public ThreadStatus call() {
-            BookKeeperJournalManager bkjm = null;
-            try {
-              bkjm = new BookKeeperJournalManager(conf, uri, nsi);
-              barrier.await();
-              bkjm.format(nsi);
-              return ThreadStatus.COMPLETED;
-            } catch (IOException ioe) {
-              LOG.info("Exception formatting ", ioe);
-              return ThreadStatus.GOODEXCEPTION;
-            } catch (InterruptedException ie) {
-              LOG.error("Interrupted. Something is broken", ie);
-              Thread.currentThread().interrupt();
-              return ThreadStatus.BADEXCEPTION;
-            } catch (Exception e) {
-              LOG.error("Some other bad exception", e);
-              return ThreadStatus.BADEXCEPTION;
-            } finally {
-              if (bkjm != null) {
-                try {
-                  bkjm.close();
-                } catch (IOException ioe) {
-                  LOG.error("Error closing journal manager", ioe);
-                }
-              }
-            }
-          }
-        });
-    }
-    ExecutorService service = Executors.newFixedThreadPool(numThreads);
-    List<Future<ThreadStatus>> statuses = service.invokeAll(threads, 60,
-                                                      TimeUnit.SECONDS);
-    int numCompleted = 0;
-    for (Future<ThreadStatus> s : statuses) {
-      assertTrue(s.isDone());
-      assertTrue("Thread threw invalid exception",
-          s.get() == ThreadStatus.COMPLETED
-          || s.get() == ThreadStatus.GOODEXCEPTION);
-      if (s.get() == ThreadStatus.COMPLETED) {
-        numCompleted++;
-      }
-    }
-    LOG.info("Completed " + numCompleted + " formats");
-    assertTrue("No thread managed to complete formatting", numCompleted > 0);
-  }
-
-  @Test(timeout = 120000)
-  public void testDefaultAckQuorum() throws Exception {
-    newBookie = bkutil.newBookie();
-    int ensembleSize = numBookies + 1;
-    int quorumSize = numBookies + 1;
-    // ensure that the journal manager has to use all bookies,
-    // so that a failure will fail the journal manager
-    Configuration conf = new Configuration();
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-        ensembleSize);
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
-        quorumSize);
-    // sets 2 secs
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
-        2);
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"), nsi);
-    bkjm.format(nsi);
-    CountDownLatch sleepLatch = new CountDownLatch(1);
-    sleepBookie(sleepLatch, newBookie);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    int numTransactions = 100;
-    for (long i = 1; i <= numTransactions; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    try {
-      out.close();
-      bkjm.finalizeLogSegment(1, numTransactions);
-
-      List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
-      bkjm.selectInputStreams(in, 1, true);
-      try {
-        assertEquals(numTransactions,
-            FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
-      } finally {
-        in.get(0).close();
-      }
-      fail("Should throw exception as not enough non-faulty bookies available!");
-    } catch (IOException ioe) {
-      // expected
-    }
-  }
-
-  /**
-   * Test ack quorum feature supported by bookkeeper. Keep ack quorum bookie
-   * alive and sleep all the other bookies. Now the client would wait for the
-   * acknowledgement from the ack size bookies and after receiving the success
-   * response will continue writing. Non ack client will hang long time to add
-   * entries.
-   */
-  @Test(timeout = 120000)
-  public void testAckQuorum() throws Exception {
-    // slow bookie
-    newBookie = bkutil.newBookie();
-    // make quorum size and ensemble size same to avoid the interleave writing
-    // of the ledger entries
-    int ensembleSize = numBookies + 1;
-    int quorumSize = numBookies + 1;
-    int ackSize = numBookies;
-    // ensure that the journal manager has to use all bookies,
-    // so that a failure will fail the journal manager
-    Configuration conf = new Configuration();
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-        ensembleSize);
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
-        quorumSize);
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ACK_QUORUM_SIZE,
-        ackSize);
-    // sets 60 minutes
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
-        3600);
-
-    NamespaceInfo nsi = newNSInfo();
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"), nsi);
-    bkjm.format(nsi);
-    CountDownLatch sleepLatch = new CountDownLatch(1);
-    sleepBookie(sleepLatch, newBookie);
-
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    int numTransactions = 100;
-    for (long i = 1; i <= numTransactions; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, numTransactions);
-
-    List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
-    bkjm.selectInputStreams(in, 1, true);
-    try {
-      assertEquals(numTransactions,
-          FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
-    } finally {
-      sleepLatch.countDown();
-      in.get(0).close();
-      bkjm.close();
-    }
-  }
-
-  /**
-   * Sleep a bookie until I count down the latch
-   *
-   * @param latch
-   *          Latch to wait on
-   * @param bookie
-   *          bookie server
-   * @throws Exception
-   */
-  private void sleepBookie(final CountDownLatch l, final BookieServer bookie)
-      throws Exception {
-
-    Thread sleeper = new Thread() {
-      public void run() {
-        try {
-          bookie.suspendProcessing();
-          l.await(60, TimeUnit.SECONDS);
-          bookie.resumeProcessing();
-        } catch (Exception e) {
-          LOG.error("Error suspending bookie", e);
-        }
-      }
-    };
-    sleeper.setName("BookieServerSleeper-" + bookie.getBookie().getId());
-    sleeper.start();
-  }
-
-
-  private String startAndFinalizeLogSegment(BookKeeperJournalManager bkjm,
-      int startTxid, int endTxid) throws IOException, KeeperException,
-      InterruptedException {
-    EditLogOutputStream out = bkjm.startLogSegment(startTxid,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long i = startTxid; i <= endTxid; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    // finalize the inprogress_1 log segment.
-    bkjm.finalizeLogSegment(startTxid, endTxid);
-    String zkpath1 = bkjm.finalizedLedgerZNode(startTxid, endTxid);
-    assertNotNull(zkc.exists(zkpath1, false));
-    assertNull(zkc.exists(bkjm.inprogressZNode(startTxid), false));
-    return zkpath1;
-  }
-}

+ 0 - 167
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperSpeculativeRead.java

@@ -1,167 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
-import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestBookKeeperSpeculativeRead {
-  private static final Log LOG = LogFactory
-      .getLog(TestBookKeeperSpeculativeRead.class);
-
-  private ZooKeeper zkc;
-  private static BKJMUtil bkutil;
-  private static int numLocalBookies = 1;
-  private static List<BookieServer> bks = new ArrayList<BookieServer>();
-
-  @BeforeClass
-  public static void setupBookkeeper() throws Exception {
-    bkutil = new BKJMUtil(1);
-    bkutil.start();
-  }
-
-  @AfterClass
-  public static void teardownBookkeeper() throws Exception {
-    bkutil.teardown();
-    for (BookieServer bk : bks) {
-      bk.shutdown();
-    }
-  }
-
-  @Before
-  public void setup() throws Exception {
-    zkc = BKJMUtil.connectZooKeeper();
-  }
-
-  @After
-  public void teardown() throws Exception {
-    zkc.close();
-  }
-
-  private NamespaceInfo newNSInfo() {
-    Random r = new Random();
-    return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
-  }
-
-  /**
-   * Test speculative read feature supported by bookkeeper. Keep one bookie
-   * alive and sleep all the other bookies. Non spec client will hang for long
-   * time to read the entries from the bookkeeper.
-   */
-  @Test(timeout = 120000)
-  public void testSpeculativeRead() throws Exception {
-    // starting 9 more servers
-    for (int i = 1; i < 10; i++) {
-      bks.add(bkutil.newBookie());
-    }
-    NamespaceInfo nsi = newNSInfo();
-    Configuration conf = new Configuration();
-    int ensembleSize = numLocalBookies + 9;
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
-        ensembleSize);
-    conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
-        ensembleSize);
-    conf.setInt(
-        BookKeeperJournalManager.BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
-        100);
-    // sets 60 minute
-    conf.setInt(
-        BookKeeperJournalManager.BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_SEC, 3600);
-    BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
-        BKJMUtil.createJournalURI("/hdfsjournal-specread"), nsi);
-    bkjm.format(nsi);
-
-    final long numTransactions = 1000;
-    EditLogOutputStream out = bkjm.startLogSegment(1,
-        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
-    for (long i = 1; i <= numTransactions; i++) {
-      FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
-      op.setTransactionId(i);
-      out.write(op);
-    }
-    out.close();
-    bkjm.finalizeLogSegment(1, numTransactions);
-
-    List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
-    bkjm.selectInputStreams(in, 1, true);
-
-    // sleep 9 bk servers. Now only one server is running and responding to the
-    // clients
-    CountDownLatch sleepLatch = new CountDownLatch(1);
-    for (final BookieServer bookie : bks) {
-      sleepBookie(sleepLatch, bookie);
-    }
-    try {
-      assertEquals(numTransactions,
-          FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
-    } finally {
-      in.get(0).close();
-      sleepLatch.countDown();
-      bkjm.close();
-    }
-  }
-
-  /**
-   * Sleep a bookie until I count down the latch
-   *
-   * @param latch
-   *          latch to wait on
-   * @param bookie
-   *          bookie server
-   * @throws Exception
-   */
-  private void sleepBookie(final CountDownLatch latch, final BookieServer bookie)
-      throws Exception {
-
-    Thread sleeper = new Thread() {
-      public void run() {
-        try {
-          bookie.suspendProcessing();
-          latch.await(2, TimeUnit.MINUTES);
-          bookie.resumeProcessing();
-        } catch (Exception e) {
-          LOG.error("Error suspending bookie", e);
-        }
-      }
-    };
-    sleeper.setName("BookieServerSleeper-" + bookie.getBookie().getId());
-    sleeper.start();
-  }
-}

+ 0 - 170
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java

@@ -1,170 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import java.io.File;
-import java.io.FileFilter;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
-import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints.SlowCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableList;
-
-public class TestBootstrapStandbyWithBKJM {
-  private static BKJMUtil bkutil;
-  protected MiniDFSCluster cluster;
-
-  @BeforeClass
-  public static void setupBookkeeper() throws Exception {
-    bkutil = new BKJMUtil(3);
-    bkutil.start();
-  }
-
-  @AfterClass
-  public static void teardownBookkeeper() throws Exception {
-    bkutil.teardown();
-  }
-
-  @After
-  public void teardown() {
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
-    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
-    conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
-        .createJournalURI("/bootstrapStandby").toString());
-    BKJMUtil.addJournalManagerDefinition(conf);
-    conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
-    conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
-        SlowCodec.class.getCanonicalName());
-    CompressionCodecFactory.setCodecClasses(conf,
-        ImmutableList.<Class> of(SlowCodec.class));
-    MiniDFSNNTopology topology = new MiniDFSNNTopology()
-        .addNameservice(new MiniDFSNNTopology.NSConf("ns1").addNN(
-            new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)).addNN(
-            new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
-    cluster = new MiniDFSCluster.Builder(conf).nnTopology(topology)
-        .numDataNodes(1).manageNameDfsSharedDirs(false).build();
-    cluster.waitActive();
-  }
-
-  /**
-   * While boostrapping, in_progress transaction entries should be skipped.
-   * Bootstrap usage for BKJM : "-force", "-nonInteractive", "-skipSharedEditsCheck"
-   */
-  @Test
-  public void testBootstrapStandbyWithActiveNN() throws Exception {
-    // make nn0 active
-    cluster.transitionToActive(0);
-   
-    // do ops and generate in-progress edit log data
-    Configuration confNN1 = cluster.getConfiguration(1);
-    DistributedFileSystem dfs = (DistributedFileSystem) HATestUtil
-        .configureFailoverFs(cluster, confNN1);
-    for (int i = 1; i <= 10; i++) {
-      dfs.mkdirs(new Path("/test" + i));
-    }
-    dfs.close();
-
-    // shutdown nn1 and delete its edit log files
-    cluster.shutdownNameNode(1);
-    deleteEditLogIfExists(confNN1);
-    cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_ENTER, true);
-    cluster.getNameNodeRpc(0).saveNamespace(0, 0);
-    cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_LEAVE, true);
-
-    // check without -skipSharedEditsCheck, Bootstrap should fail for BKJM
-    // immediately after saveNamespace
-    int rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive" },
-      confNN1);
-    Assert.assertEquals("Mismatches return code", 6, rc);
-
-    // check with -skipSharedEditsCheck
-    rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive",
-        "-skipSharedEditsCheck" }, confNN1);
-    Assert.assertEquals("Mismatches return code", 0, rc);
-
-    // Checkpoint as fast as we can, in a tight loop.
-    confNN1.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 1);
-    cluster.restartNameNode(1);
-    cluster.transitionToStandby(1);
-   
-    NameNode nn0 = cluster.getNameNode(0);
-    HATestUtil.waitForStandbyToCatchUp(nn0, cluster.getNameNode(1));
-    long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
-        .getFSImage().getMostRecentCheckpointTxId();
-    HATestUtil.waitForCheckpoint(cluster, 1,
-        ImmutableList.of((int) expectedCheckpointTxId));
-
-    // Should have copied over the namespace
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of((int) expectedCheckpointTxId));
-    FSImageTestUtil.assertNNFilesMatch(cluster);
-  }
-
-  private void deleteEditLogIfExists(Configuration confNN1) {
-    String editDirs = confNN1.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
-    String[] listEditDirs = StringUtils.split(editDirs, ',');
-    Assert.assertTrue("Wrong edit directory path!", listEditDirs.length > 0);
-
-    for (String dir : listEditDirs) {
-      File curDir = new File(dir, "current");
-      File[] listFiles = curDir.listFiles(new FileFilter() {
-        @Override
-        public boolean accept(File f) {
-          if (!f.getName().startsWith("edits")) {
-            return true;
-          }
-          return false;
-        }
-      });
-      if (listFiles != null && listFiles.length > 0) {
-        for (File file : listFiles) {
-          Assert.assertTrue("Failed to delete edit files!", file.delete());
-        }
-      }
-    }
-  }
-}

+ 0 - 160
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestCurrentInprogress.java

@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.contrib.bkjournal;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Tests that read, update, clear api from CurrentInprogress
- */
-public class TestCurrentInprogress {
-  private static final Log LOG = LogFactory.getLog(TestCurrentInprogress.class);
-  private static final String CURRENT_NODE_PATH = "/test";
-  private static final String HOSTPORT = "127.0.0.1:2181";
-  private static final int CONNECTION_TIMEOUT = 30000;
-  private static NIOServerCnxnFactory serverFactory;
-  private static ZooKeeperServer zks;
-  private static ZooKeeper zkc;
-  private static int ZooKeeperDefaultPort = 2181;
-  private static File zkTmpDir;
-
-  private static ZooKeeper connectZooKeeper(String ensemble)
-      throws IOException, KeeperException, InterruptedException {
-    final CountDownLatch latch = new CountDownLatch(1);
-
-    ZooKeeper zkc = new ZooKeeper(HOSTPORT, 3600, new Watcher() {
-      public void process(WatchedEvent event) {
-        if (event.getState() == Watcher.Event.KeeperState.SyncConnected) {
-          latch.countDown();
-        }
-      }
-    });
-    if (!latch.await(10, TimeUnit.SECONDS)) {
-      throw new IOException("Zookeeper took too long to connect");
-    }
-    return zkc;
-  }
-
-  @BeforeClass
-  public static void setupZooKeeper() throws Exception {
-    LOG.info("Starting ZK server");
-    zkTmpDir = File.createTempFile("zookeeper", "test");
-    zkTmpDir.delete();
-    zkTmpDir.mkdir();
-    try {
-      zks = new ZooKeeperServer(zkTmpDir, zkTmpDir, ZooKeeperDefaultPort);
-      serverFactory = new NIOServerCnxnFactory();
-      serverFactory.configure(new InetSocketAddress(ZooKeeperDefaultPort), 10);
-      serverFactory.startup(zks);
-    } catch (Exception e) {
-      LOG.error("Exception while instantiating ZooKeeper", e);
-    }
-    boolean b = LocalBookKeeper.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT);
-    LOG.debug("ZooKeeper server up: " + b);
-  }
-
-  @AfterClass
-  public static void shutDownServer() {
-    if (null != zks) {
-      zks.shutdown();
-    }
-    zkTmpDir.delete();
-  }
-
-  @Before
-  public void setup() throws Exception {
-    zkc = connectZooKeeper(HOSTPORT);
-  }
-
-  @After
-  public void teardown() throws Exception {
-    if (null != zkc) {
-      zkc.close();
-    }
-
-  }
-
-  /**
-   * Tests that read should be able to read the data which updated with update
-   * api
-   */
-  @Test
-  public void testReadShouldReturnTheZnodePathAfterUpdate() throws Exception {
-    String data = "inprogressNode";
-    CurrentInprogress ci = new CurrentInprogress(zkc, CURRENT_NODE_PATH);
-    ci.init();
-    ci.update(data);
-    String inprogressNodePath = ci.read();
-    assertEquals("Not returning inprogressZnode", "inprogressNode",
-        inprogressNodePath);
-  }
-
-  /**
-   * Tests that read should return null if we clear the updated data in
-   * CurrentInprogress node
-   */
-  @Test
-  public void testReadShouldReturnNullAfterClear() throws Exception {
-    CurrentInprogress ci = new CurrentInprogress(zkc, CURRENT_NODE_PATH);
-    ci.init();
-    ci.update("myInprogressZnode");
-    ci.read();
-    ci.clear();
-    String inprogressNodePath = ci.read();
-    assertEquals("Expecting null to be return", null, inprogressNodePath);
-  }
-
-  /**
-   * Tests that update should throw IOE, if version number modifies between read
-   * and update
-   */
-  @Test(expected = IOException.class)
-  public void testUpdateShouldFailWithIOEIfVersionNumberChangedAfterRead()
-      throws Exception {
-    CurrentInprogress ci = new CurrentInprogress(zkc, CURRENT_NODE_PATH);
-    ci.init();
-    ci.update("myInprogressZnode");
-    assertEquals("Not returning myInprogressZnode", "myInprogressZnode", ci
-        .read());
-    // Updating data in-between to change the data to change the version number
-    ci.update("YourInprogressZnode");
-    ci.update("myInprogressZnode");
-  }
-
-}

+ 0 - 55
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties

@@ -1,55 +0,0 @@
-#
-# 
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-# 
-#   http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-# 
-#
-
-#
-# Bookkeeper Journal Logging Configuration
-#
-
-# Format is "<default threshold> (, <appender>)+
-
-# DEFAULT: console appender only
-log4j.rootLogger=DEBUG, CONSOLE
-
-# Example with rolling log file
-#log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE
-
-# Example with rolling log file and tracing
-#log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE
-
-#
-# Log INFO level and above messages to the console
-#
-log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
-log4j.appender.CONSOLE.Threshold=INFO
-log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
-log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-#
-# Add ROLLINGFILE to rootLogger to get log file output
-#    Log DEBUG level and above messages to a log file
-log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.ROLLINGFILE.Threshold=DEBUG
-log4j.appender.ROLLINGFILE.File=hdfs-namenode.log
-log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
-log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-# Max log file size of 10MB
-log4j.appender.ROLLINGFILE.MaxFileSize=10MB

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java

@@ -29,6 +29,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
+import static org.apache.hadoop.security.SecurityUtil.buildTokenService;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -56,7 +57,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
@@ -281,8 +281,7 @@ public class HAUtil {
         // exposed to the user via UGI.getCredentials(), otherwise these
         // cloned tokens may be inadvertently propagated to jobs
         Token<DelegationTokenIdentifier> specificToken =
-            new Token.PrivateToken<DelegationTokenIdentifier>(haToken);
-        SecurityUtil.setTokenService(specificToken, singleNNAddr);
+            haToken.privateClone(buildTokenService(singleNNAddr));
         Text alias = new Text(
             HAUtilClient.buildTokenServicePrefixForLogicalUri(
                 HdfsConstants.HDFS_URI_SCHEME)

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockPoolTokenSecretManager.java

@@ -49,7 +49,8 @@ public class BlockPoolTokenSecretManager extends
     map.put(bpid, secretMgr);
   }
 
-  synchronized BlockTokenSecretManager get(String bpid) {
+  @VisibleForTesting
+  public synchronized BlockTokenSecretManager get(String bpid) {
     BlockTokenSecretManager secretMgr = map.get(bpid);
     if (secretMgr == null) {
       throw new IllegalArgumentException("Block pool " + bpid

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java

@@ -435,6 +435,12 @@ public class BlockTokenSecretManager extends
     allKeys.clear();
   }
 
+  @VisibleForTesting
+  public synchronized boolean hasKey(int keyId) {
+    BlockKey key = allKeys.get(keyId);
+    return key != null;
+  }
+
   @VisibleForTesting
   public synchronized int getSerialNoForTesting() {
     return serialNo;

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

@@ -354,7 +354,7 @@ public class Dispatcher {
         target.getDDatanode().setHasSuccess();
         LOG.info("Successfully moved " + this);
       } catch (IOException e) {
-        LOG.warn("Failed to move " + this + ": " + e.getMessage());
+        LOG.warn("Failed to move " + this, e);
         target.getDDatanode().setHasFailure();
         // Proxy or target may have some issues, delay before using these nodes
         // further in order to avoid a potential storm of "threads quota

+ 55 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -83,6 +83,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
@@ -102,6 +103,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.util.FoldedTreeSet;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.server.namenode.CacheManager;
 
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 
@@ -786,12 +788,13 @@ public class BlockManager implements BlockStatsMXBean {
    * 
    * @param bc block collection
    * @param commitBlock - contains client reported block length and generation
+   * @param iip - INodes in path to bc
    * @return true if the last block is changed to committed state.
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
   public boolean commitOrCompleteLastBlock(BlockCollection bc,
-      Block commitBlock) throws IOException {
+      Block commitBlock, INodesInPath iip) throws IOException {
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
     BlockInfo lastBlock = bc.getLastBlock();
@@ -811,7 +814,7 @@ public class BlockManager implements BlockStatsMXBean {
       if (committed) {
         addExpectedReplicasToPending(lastBlock);
       }
-      completeBlock(lastBlock, false);
+      completeBlock(lastBlock, iip, false);
     }
     return committed;
   }
@@ -841,11 +844,15 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * Convert a specified block of the file to a complete block.
+   * @param curBlock - block to be completed
+   * @param iip - INodes in path to file containing curBlock; if null,
+   *              this will be resolved internally
+   * @param force - force completion of the block
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private void completeBlock(BlockInfo curBlock, boolean force)
-      throws IOException {
+  private void completeBlock(BlockInfo curBlock, INodesInPath iip,
+      boolean force) throws IOException {
     if (curBlock.isComplete()) {
       return;
     }
@@ -860,7 +867,8 @@ public class BlockManager implements BlockStatsMXBean {
           "Cannot complete block: block has not been COMMITTED by the client");
     }
 
-    curBlock.convertToCompleteBlock();
+    convertToCompleteBlock(curBlock, iip);
+
     // Since safe-mode only counts complete blocks, and we now have
     // one more complete block, we need to adjust the total up, and
     // also count it as safe, if we have at least the minimum replica
@@ -874,6 +882,22 @@ public class BlockManager implements BlockStatsMXBean {
         curBlock);
   }
 
+  /**
+   * Convert a specified block of the file to a complete block.
+   * Skips validity checking and safe mode block total updates; use
+   * {@link BlockManager#completeBlock} to include these.
+   * @param curBlock - block to be completed
+   * @param iip - INodes in path to file containing curBlock; if null,
+   *              this will be resolved internally
+   * @throws IOException if the block does not have at least a minimal number
+   * of replicas reported from data-nodes.
+   */
+  private void convertToCompleteBlock(BlockInfo curBlock, INodesInPath iip)
+      throws IOException {
+    curBlock.convertToCompleteBlock();
+    namesystem.getFSDirectory().updateSpaceForCompleteBlock(curBlock, iip);
+  }
+
   /**
    * Force the given block in the given file to be marked as complete,
    * regardless of whether enough replicas are present. This is necessary
@@ -881,7 +905,7 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public void forceCompleteBlock(final BlockInfo block) throws IOException {
     block.commitBlock(block);
-    completeBlock(block, true);
+    completeBlock(block, null, true);
   }
 
   /**
@@ -1036,7 +1060,8 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     // get block locations
-    final int numCorruptNodes = countNodes(blk).corruptReplicas();
+    NumberReplicas numReplicas = countNodes(blk);
+    final int numCorruptNodes = numReplicas.corruptReplicas();
     final int numCorruptReplicas = corruptReplicas.numCorruptReplicas(blk);
     if (numCorruptNodes != numCorruptReplicas) {
       LOG.warn("Inconsistent number of corrupt replicas for "
@@ -1045,8 +1070,14 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     final int numNodes = blocksMap.numNodes(blk);
-    final boolean isCorrupt = numCorruptReplicas != 0 &&
-        numCorruptReplicas == numNodes;
+    final boolean isCorrupt;
+    if (blk.isStriped()) {
+      BlockInfoStriped sblk = (BlockInfoStriped) blk;
+      isCorrupt = numCorruptReplicas != 0 &&
+          numReplicas.liveReplicas() < sblk.getRealDataBlockNum();
+    } else {
+      isCorrupt = numCorruptReplicas != 0 && numCorruptReplicas == numNodes;
+    }
     final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptReplicas;
     DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
     final byte[] blockIndices = blk.isStriped() ? new byte[numMachines] : null;
@@ -1122,9 +1153,16 @@ public class BlockManager implements BlockStatsMXBean {
             fileSizeExcludeBlocksUnderConstruction, mode);
         isComplete = true;
       }
-      return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction,
+      LocatedBlocks locations = new LocatedBlocks(
+          fileSizeExcludeBlocksUnderConstruction,
           isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
           ecPolicy);
+      // Set caching information for the located blocks.
+      CacheManager cm = namesystem.getCacheManager();
+      if (cm != null) {
+        cm.setCachedLocations(locations);
+      }
+      return locations;
     }
   }
 
@@ -2910,7 +2948,7 @@ public class BlockManager implements BlockStatsMXBean {
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
         && hasMinStorage(storedBlock, numCurrentReplica)) {
-      completeBlock(storedBlock, false);
+      completeBlock(storedBlock, null, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that.
@@ -2985,7 +3023,7 @@ public class BlockManager implements BlockStatsMXBean {
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
         hasMinStorage(storedBlock, numLiveReplicas)) {
       addExpectedReplicasToPending(storedBlock);
-      completeBlock(storedBlock, false);
+      completeBlock(storedBlock, null, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that
@@ -3982,13 +4020,15 @@ public class BlockManager implements BlockStatsMXBean {
         return;
       }
       NumberReplicas repl = countNodes(block);
+      int pendingNum = pendingReconstruction.getNumReplicas(block);
       int curExpectedReplicas = getRedundancy(block);
-      if (isNeededReconstruction(block, repl.liveReplicas())) {
-        neededReconstruction.update(block, repl.liveReplicas(),
+      if (!hasEnoughEffectiveReplicas(block, repl, pendingNum,
+          curExpectedReplicas)) {
+        neededReconstruction.update(block, repl.liveReplicas() + pendingNum,
             repl.readOnlyReplicas(), repl.decommissionedAndDecommissioning(),
             curExpectedReplicas, curReplicasDelta, expectedReplicasDelta);
       } else {
-        int oldReplicas = repl.liveReplicas()-curReplicasDelta;
+        int oldReplicas = repl.liveReplicas() + pendingNum - curReplicasDelta;
         int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
         neededReconstruction.remove(block, oldReplicas, repl.readOnlyReplicas(),
             repl.decommissionedAndDecommissioning(), oldExpectedReplicas);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -49,8 +49,9 @@ import com.google.common.annotations.VisibleForTesting;
 public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
   private static final String enableDebugLogging =
-    "For more information, please enable DEBUG log level on "
-    + BlockPlacementPolicy.class.getName();
+      "For more information, please enable DEBUG log level on "
+          + BlockPlacementPolicy.class.getName() + " and "
+          + NetworkTopology.class.getName();
 
   private static final ThreadLocal<StringBuilder> debugLoggingBuilder
       = new ThreadLocal<StringBuilder>() {

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -2779,6 +2779,11 @@ public class DataNode extends ReconfigurableBase
     return directoryScanner;
   }
 
+  @VisibleForTesting
+  public BlockPoolTokenSecretManager getBlockPoolTokenSecretManager() {
+    return blockPoolTokenSecretManager;
+  }
+
   public static void secureMain(String args[], SecureResources resources) {
     int errorCode = 0;
     try {

+ 143 - 67
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java

@@ -22,6 +22,8 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi
+    .FsVolumeReferences;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -41,6 +43,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.LinkedList;
@@ -192,7 +195,30 @@ public class DiskBalancer {
   }
 
   /**
-   * Returns the Current Work Status of a submitted Plan.
+   * Get FsVolume by volume UUID.
+   * @param fsDataset
+   * @param volUuid
+   * @return FsVolumeSpi
+   */
+  private static FsVolumeSpi getFsVolume(final FsDatasetSpi<?> fsDataset,
+      final String volUuid) {
+    FsVolumeSpi fsVolume = null;
+    try (FsVolumeReferences volumeReferences =
+           fsDataset.getFsVolumeReferences()) {
+      for (int i = 0; i < volumeReferences.size(); i++) {
+        if (volumeReferences.get(i).getStorageID().equals(volUuid)) {
+          fsVolume = volumeReferences.get(i);
+          break;
+        }
+      }
+    } catch (IOException e) {
+      LOG.warn("Disk Balancer - Error when closing volume references: ", e);
+    }
+    return fsVolume;
+  }
+
+  /**
+   * Returns the current work status of a previously submitted Plan.
    *
    * @return DiskBalancerWorkStatus.
    * @throws DiskBalancerException
@@ -214,8 +240,8 @@ public class DiskBalancer {
       for (Map.Entry<VolumePair, DiskBalancerWorkItem> entry :
           workMap.entrySet()) {
         DiskBalancerWorkEntry workEntry = new DiskBalancerWorkEntry(
-            entry.getKey().getSource().getBasePath(),
-            entry.getKey().getDest().getBasePath(),
+            entry.getKey().getSourceVolBasePath(),
+            entry.getKey().getDestVolBasePath(),
             entry.getValue());
         status.addWorkEntry(workEntry);
       }
@@ -269,12 +295,7 @@ public class DiskBalancer {
     lock.lock();
     try {
       checkDiskBalancerEnabled();
-      Map<String, String> pathMap = new HashMap<>();
-      Map<String, FsVolumeSpi> volMap = getStorageIDToVolumeMap();
-      for (Map.Entry<String, FsVolumeSpi> entry : volMap.entrySet()) {
-        pathMap.put(entry.getKey(), entry.getValue().getBasePath());
-      }
-      return JsonUtil.toJsonString(pathMap);
+      return JsonUtil.toJsonString(getStorageIDToVolumeBasePathMap());
     } catch (DiskBalancerException ex) {
       throw ex;
     } catch (IOException e) {
@@ -434,47 +455,52 @@ public class DiskBalancer {
 
     // Cleanup any residual work in the map.
     workMap.clear();
-    Map<String, FsVolumeSpi> pathMap = getStorageIDToVolumeMap();
+    Map<String, String> storageIDToVolBasePathMap =
+        getStorageIDToVolumeBasePathMap();
 
     for (Step step : plan.getVolumeSetPlans()) {
-      String sourceuuid = step.getSourceVolume().getUuid();
-      String destinationuuid = step.getDestinationVolume().getUuid();
-
-      FsVolumeSpi sourceVol = pathMap.get(sourceuuid);
-      if (sourceVol == null) {
-        LOG.error("Disk Balancer - Unable to find source volume. submitPlan " +
-            "failed.");
-        throw new DiskBalancerException("Unable to find source volume.",
+      String sourceVolUuid = step.getSourceVolume().getUuid();
+      String destVolUuid = step.getDestinationVolume().getUuid();
+
+      String sourceVolBasePath = storageIDToVolBasePathMap.get(sourceVolUuid);
+      if (sourceVolBasePath == null) {
+        final String errMsg = "Disk Balancer - Unable to find volume: "
+            + step.getSourceVolume().getPath() + ". SubmitPlan failed.";
+        LOG.error(errMsg);
+        throw new DiskBalancerException(errMsg,
             DiskBalancerException.Result.INVALID_VOLUME);
       }
 
-      FsVolumeSpi destVol = pathMap.get(destinationuuid);
-      if (destVol == null) {
-        LOG.error("Disk Balancer - Unable to find destination volume. " +
-            "submitPlan failed.");
-        throw new DiskBalancerException("Unable to find destination volume.",
+      String destVolBasePath = storageIDToVolBasePathMap.get(destVolUuid);
+      if (destVolBasePath == null) {
+        final String errMsg = "Disk Balancer - Unable to find volume: "
+            + step.getDestinationVolume().getPath() + ". SubmitPlan failed.";
+        LOG.error(errMsg);
+        throw new DiskBalancerException(errMsg,
             DiskBalancerException.Result.INVALID_VOLUME);
       }
-      createWorkPlan(sourceVol, destVol, step);
+      VolumePair volumePair = new VolumePair(sourceVolUuid,
+          sourceVolBasePath, destVolUuid, destVolBasePath);
+      createWorkPlan(volumePair, step);
     }
   }
 
   /**
-   * Returns a path to Volume Map.
+   * Returns volume UUID to volume base path map.
    *
    * @return Map
    * @throws DiskBalancerException
    */
-  private Map<String, FsVolumeSpi> getStorageIDToVolumeMap()
+  private Map<String, String> getStorageIDToVolumeBasePathMap()
       throws DiskBalancerException {
-    Map<String, FsVolumeSpi> pathMap = new HashMap<>();
+    Map<String, String> storageIDToVolBasePathMap = new HashMap<>();
     FsDatasetSpi.FsVolumeReferences references;
     try {
       try(AutoCloseableLock lock = this.dataset.acquireDatasetLock()) {
         references = this.dataset.getFsVolumeReferences();
         for (int ndx = 0; ndx < references.size(); ndx++) {
           FsVolumeSpi vol = references.get(ndx);
-          pathMap.put(vol.getStorageID(), vol);
+          storageIDToVolBasePathMap.put(vol.getStorageID(), vol.getBasePath());
         }
         references.close();
       }
@@ -483,7 +509,7 @@ public class DiskBalancer {
       throw new DiskBalancerException("Internal error", ex,
           DiskBalancerException.Result.INTERNAL_ERROR);
     }
-    return pathMap;
+    return storageIDToVolBasePathMap;
   }
 
   /**
@@ -513,26 +539,24 @@ public class DiskBalancer {
 
   /**
    * Insert work items to work map.
-   *
-   * @param source - Source vol
-   * @param dest   - destination volume
-   * @param step   - Move Step
+   * @param volumePair - VolumePair
+   * @param step - Move Step
    */
-  private void createWorkPlan(FsVolumeSpi source, FsVolumeSpi dest,
-                              Step step) throws DiskBalancerException {
-
-    if (source.getStorageID().equals(dest.getStorageID())) {
-      LOG.info("Disk Balancer - source & destination volumes are same.");
-      throw new DiskBalancerException("source and destination volumes are " +
-          "same.", DiskBalancerException.Result.INVALID_MOVE);
+  private void createWorkPlan(final VolumePair volumePair, Step step)
+      throws DiskBalancerException {
+    if (volumePair.getSourceVolUuid().equals(volumePair.getDestVolUuid())) {
+      final String errMsg = "Disk Balancer - Source and destination volumes " +
+          "are same: " + volumePair.getSourceVolUuid();
+      LOG.warn(errMsg);
+      throw new DiskBalancerException(errMsg,
+          DiskBalancerException.Result.INVALID_MOVE);
     }
-    VolumePair pair = new VolumePair(source, dest);
     long bytesToMove = step.getBytesToMove();
     // In case we have a plan with more than
-    // one line of same <source, dest>
+    // one line of same VolumePair
     // we compress that into one work order.
-    if (workMap.containsKey(pair)) {
-      bytesToMove += workMap.get(pair).getBytesToCopy();
+    if (workMap.containsKey(volumePair)) {
+      bytesToMove += workMap.get(volumePair).getBytesToCopy();
     }
 
     DiskBalancerWorkItem work = new DiskBalancerWorkItem(bytesToMove, 0);
@@ -542,7 +566,7 @@ public class DiskBalancer {
     work.setBandwidth(step.getBandwidth());
     work.setTolerancePercent(step.getTolerancePercent());
     work.setMaxDiskErrors(step.getMaxDiskErrors());
-    workMap.put(pair, work);
+    workMap.put(volumePair, work);
   }
 
   /**
@@ -591,39 +615,63 @@ public class DiskBalancer {
   }
 
   /**
-   * Holds references to actual volumes that we will be operating against.
+   * Holds source and dest volumes UUIDs and their BasePaths
+   * that disk balancer will be operating against.
    */
   public static class VolumePair {
-    private final FsVolumeSpi source;
-    private final FsVolumeSpi dest;
+    private final String sourceVolUuid;
+    private final String destVolUuid;
+    private final String sourceVolBasePath;
+    private final String destVolBasePath;
 
     /**
      * Constructs a volume pair.
+     * @param sourceVolUuid     - Source Volume
+     * @param sourceVolBasePath - Source Volume Base Path
+     * @param destVolUuid       - Destination Volume
+     * @param destVolBasePath   - Destination Volume Base Path
+     */
+    public VolumePair(final String sourceVolUuid,
+        final String sourceVolBasePath, final String destVolUuid,
+        final String destVolBasePath) {
+      this.sourceVolUuid = sourceVolUuid;
+      this.sourceVolBasePath = sourceVolBasePath;
+      this.destVolUuid = destVolUuid;
+      this.destVolBasePath = destVolBasePath;
+    }
+
+    /**
+     * Gets source volume UUID.
      *
-     * @param source - Source Volume
-     * @param dest   - Destination Volume
+     * @return UUID String
      */
-    public VolumePair(FsVolumeSpi source, FsVolumeSpi dest) {
-      this.source = source;
-      this.dest = dest;
+    public String getSourceVolUuid() {
+      return sourceVolUuid;
     }
 
     /**
-     * gets source volume.
+     * Gets source volume base path.
+     * @return String
+     */
+    public String getSourceVolBasePath() {
+      return sourceVolBasePath;
+    }
+    /**
+     * Gets destination volume UUID.
      *
-     * @return volume
+     * @return UUID String
      */
-    public FsVolumeSpi getSource() {
-      return source;
+    public String getDestVolUuid() {
+      return destVolUuid;
     }
 
     /**
-     * Gets Destination volume.
+     * Gets desitnation volume base path.
      *
-     * @return volume.
+     * @return String
      */
-    public FsVolumeSpi getDest() {
-      return dest;
+    public String getDestVolBasePath() {
+      return destVolBasePath;
     }
 
     @Override
@@ -636,13 +684,21 @@ public class DiskBalancer {
       }
 
       VolumePair that = (VolumePair) o;
-      return source.equals(that.source) && dest.equals(that.dest);
+      return sourceVolUuid.equals(that.sourceVolUuid)
+          && sourceVolBasePath.equals(that.sourceVolBasePath)
+          && destVolUuid.equals(that.destVolUuid)
+          && destVolBasePath.equals(that.destVolBasePath);
     }
 
     @Override
     public int hashCode() {
-      int result = source.getBasePath().hashCode();
-      result = 31 * result + dest.getBasePath().hashCode();
+      final int primeNum = 31;
+      final List<String> volumeStrList = Arrays.asList(sourceVolUuid,
+          sourceVolBasePath, destVolUuid, destVolBasePath);
+      int result = 1;
+      for (String str : volumeStrList) {
+        result = (result * primeNum) + str.hashCode();
+      }
       return result;
     }
   }
@@ -932,8 +988,28 @@ public class DiskBalancer {
      */
     @Override
     public void copyBlocks(VolumePair pair, DiskBalancerWorkItem item) {
-      FsVolumeSpi source = pair.getSource();
-      FsVolumeSpi dest = pair.getDest();
+      String sourceVolUuid = pair.getSourceVolUuid();
+      String destVolUuuid = pair.getDestVolUuid();
+
+      // When any of the DiskBalancerWorkItem volumes are not
+      // available, return after setting error in item.
+      FsVolumeSpi source = getFsVolume(this.dataset, sourceVolUuid);
+      if (source == null) {
+        final String errMsg = "Disk Balancer - Unable to find source volume: "
+            + pair.getDestVolBasePath();
+        LOG.error(errMsg);
+        item.setErrMsg(errMsg);
+        return;
+      }
+      FsVolumeSpi dest = getFsVolume(this.dataset, destVolUuuid);
+      if (dest == null) {
+        final String errMsg = "Disk Balancer - Unable to find dest volume: "
+            + pair.getDestVolBasePath();
+        LOG.error(errMsg);
+        item.setErrMsg(errMsg);
+        return;
+      }
+
       List<FsVolumeSpi.BlockIterator> poolIters = new LinkedList<>();
       startTime = Time.now();
       item.setStartTime(startTime);
@@ -977,7 +1053,7 @@ public class DiskBalancer {
             // we are not able to find any blocks to copy.
             if (block == null) {
               LOG.error("No source blocks, exiting the copy. Source: {}, " +
-                  "dest:{}", source.getBasePath(), dest.getBasePath());
+                  "Dest:{}", source.getBasePath(), dest.getBasePath());
               this.setExitFlag();
               continue;
             }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java

@@ -122,7 +122,7 @@ class StripedBlockReader {
           "", newConnectedPeer(block, dnAddr, blockToken, source), source,
           null, stripedReader.getCachingStrategy(), datanode.getTracer(), -1);
     } catch (IOException e) {
-      LOG.debug("Exception while creating remote block reader, datanode {}",
+      LOG.info("Exception while creating remote block reader, datanode {}",
           source, e);
       return null;
     }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -769,7 +770,7 @@ class BlockPoolSlice {
 
   private boolean readReplicasFromCache(ReplicaMap volumeMap,
       final RamDiskReplicaTracker lazyWriteReplicaMap) {
-    ReplicaMap tmpReplicaMap = new ReplicaMap(this);
+    ReplicaMap tmpReplicaMap = new ReplicaMap(new AutoCloseableLock());
     File replicaFile = new File(currentDir, REPLICA_CACHE_FILE);
     // Check whether the file exists or not.
     if (!replicaFile.exists()) {

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