Pārlūkot izejas kodu

Merge branch 'trunk' into HDFS-7240

Anu Engineer 8 gadi atpakaļ
vecāks
revīzija
7d70e57a13
100 mainītis faili ar 2954 papildinājumiem un 5935 dzēšanām
  1. 2 0
      .gitignore
  2. 1 6
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
  3. 1 1
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
  4. 99 122
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZKSignerSecretProvider.java
  5. 5 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  6. 1 0
      hadoop-common-project/hadoop-common/pom.xml
  7. 14 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java
  8. 234 73
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  9. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
  10. 48 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.java
  11. 3 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
  12. 21 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
  13. 9 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  14. 32 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
  15. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
  16. 22 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
  17. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntryScope.java
  18. 22 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntryType.java
  19. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
  20. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
  21. 16 14
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
  22. 88 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
  23. 40 23
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  24. 4 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  25. 6 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
  26. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  27. 2 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  28. 118 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosAuthException.java
  29. 46 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UGIExceptionMessages.java
  30. 96 107
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  31. 52 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java
  32. 3 3
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  33. 3 1
      hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
  34. 2 1
      hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
  35. 58 19
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  36. 119 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
  37. 140 24
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  38. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextURIBase.java
  39. 75 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java
  40. 4 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemInitialization.java
  41. 4 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
  42. 33 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java
  43. 39 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
  44. 21 13
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
  45. 87 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  46. 4 29
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  47. 397 268
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  48. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  49. 30 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  50. 17 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  51. 55 33
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
  52. 20 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java
  53. 0 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  54. 7 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
  55. 4 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
  56. 1 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
  57. 0 66
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/README.txt
  58. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/dev-support/findbugsExcludeFile.xml
  59. 0 175
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
  60. 0 264
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
  61. 0 188
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java
  62. 0 893
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  63. 0 160
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/CurrentInprogress.java
  64. 0 217
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
  65. 0 103
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/MaxTxId.java
  66. 0 184
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/BKJMUtil.java
  67. 0 414
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
  68. 0 174
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperConfiguration.java
  69. 0 92
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
  70. 0 109
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
  71. 0 984
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java
  72. 0 167
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperSpeculativeRead.java
  73. 0 170
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java
  74. 0 160
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestCurrentInprogress.java
  75. 0 55
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
  76. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  77. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  78. 29 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  79. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  80. 28 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
  81. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  82. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  83. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  84. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
  85. 12 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  86. 21 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  87. 6 314
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  88. 12 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
  89. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
  90. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  91. 27 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
  92. 13 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
  93. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
  94. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  95. 228 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
  96. 23 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
  97. 67 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  98. 214 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  99. 65 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
  100. 22 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java

+ 2 - 0
.gitignore

@@ -31,3 +31,5 @@ hadoop-tools/hadoop-aws/src/test/resources/auth-keys.xml
 hadoop-tools/hadoop-aws/src/test/resources/contract-test-options.xml
 hadoop-tools/hadoop-azure/src/test/resources/azure-auth-keys.xml
 patchprocess/
+hadoop-tools/hadoop-aliyun/src/test/resources/auth-keys.xml
+hadoop-tools/hadoop-aliyun/src/test/resources/contract-test-options.xml

+ 1 - 6
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java

@@ -343,8 +343,6 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
       authorization = authorization.substring(KerberosAuthenticator.NEGOTIATE.length()).trim();
       final Base64 base64 = new Base64(0);
       final byte[] clientToken = base64.decode(authorization);
-      final String serverName = InetAddress.getByName(request.getServerName())
-                                           .getCanonicalHostName();
       try {
         token = Subject.doAs(serverSubject, new PrivilegedExceptionAction<AuthenticationToken>() {
 
@@ -354,10 +352,7 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
             GSSContext gssContext = null;
             GSSCredential gssCreds = null;
             try {
-              gssCreds = gssManager.createCredential(
-                  gssManager.createName(
-                      KerberosUtil.getServicePrincipal("HTTP", serverName),
-                      KerberosUtil.getOidInstance("NT_GSS_KRB5_PRINCIPAL")),
+              gssCreds = gssManager.createCredential(null,
                   GSSCredential.INDEFINITE_LIFETIME,
                   new Oid[]{
                     KerberosUtil.getOidInstance("GSS_SPNEGO_MECH_OID"),

+ 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>

+ 1 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -235,6 +235,7 @@
     <dependency>
       <groupId>com.jcraft</groupId>
       <artifactId>jsch</artifactId>
+      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.curator</groupId>

+ 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;
 

+ 234 - 73
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>
@@ -942,10 +943,15 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    *
    * If var is unbounded the current state of expansion "prefix${var}suffix" is
    * returned.
-   *
-   * If a cycle is detected: replacing var1 requires replacing var2 ... requires
-   * replacing var1, i.e., the cycle is shorter than
-   * {@link Configuration#MAX_SUBST} then the original expr is returned.
+   * <p>
+   * This function also detects self-referential substitutions, i.e.
+   * <pre>
+   *   {@code
+   *   foo.bar = ${foo.bar}
+   *   }
+   * </pre>
+   * If a cycle is detected then the original expr is returned. Loops
+   * involving multiple substitutions are not detected.
    *
    * @param expr the literal value of a config key
    * @return null if expr is null, otherwise the value resulting from expanding
@@ -958,7 +964,6 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       return null;
     }
     String eval = expr;
-    Set<String> evalSet = null;
     for(int s = 0; s < MAX_SUBST; s++) {
       final int[] varBounds = findSubVariable(eval);
       if (varBounds[SUB_START_IDX] == -1) {
@@ -1003,15 +1008,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         return eval; // return literal ${var}: var is unbound
       }
 
-      // prevent recursive resolution
-      //
       final int dollar = varBounds[SUB_START_IDX] - "${".length();
       final int afterRightBrace = varBounds[SUB_END_IDX] + "}".length();
       final String refVar = eval.substring(dollar, afterRightBrace);
-      if (evalSet == null) {
-        evalSet = new HashSet<String>();
-      }
-      if (!evalSet.add(refVar)) {
+
+      // detect self-referential values
+      if (val.contains(refVar)) {
         return expr; // return original expression if there is a loop
       }
 
@@ -2834,14 +2836,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 +2886,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 +3073,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() {

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java

@@ -177,7 +177,8 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed {
           // update the used variable
           spaceUsed.refresh();
         } catch (InterruptedException e) {
-          LOG.warn("Thread Interrupted waiting to refresh disk information", e);
+          LOG.warn("Thread Interrupted waiting to refresh disk information: "
+              + e.getMessage());
           Thread.currentThread().interrupt();
         }
       }

+ 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();
+  }
 }

+ 9 - 1
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;

+ 32 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java

@@ -36,15 +36,25 @@ public abstract class TrashPolicy extends Configured {
   protected Path trash; // path to trash directory
   protected long deletionInterval; // deletion interval for Emptier
 
+  /**
+   * Used to setup the trash policy. Must be implemented by all TrashPolicy
+   * implementations.
+   * @param conf the configuration to be used
+   * @param fs the filesystem to be used
+   * @param home the home directory
+   * @deprecated Use {@link #initialize(Configuration, FileSystem)} instead.
+   */
+  @Deprecated
+  public abstract void initialize(Configuration conf, FileSystem fs, Path home);
+
   /**
    * Used to setup the trash policy. Must be implemented by all TrashPolicy
    * implementations. Different from initialize(conf, fs, home), this one does
    * not assume trash always under /user/$USER due to HDFS encryption zone.
    * @param conf the configuration to be used
    * @param fs the filesystem to be used
-   * @throws IOException
    */
-  public void initialize(Configuration conf, FileSystem fs) throws IOException{
+  public void initialize(Configuration conf, FileSystem fs) {
     throw new UnsupportedOperationException();
   }
 
@@ -99,6 +109,25 @@ public abstract class TrashPolicy extends Configured {
    */
   public abstract Runnable getEmptier() throws IOException;
 
+  /**
+   * Get an instance of the configured TrashPolicy based on the value
+   * of the configuration parameter fs.trash.classname.
+   *
+   * @param conf the configuration to be used
+   * @param fs the file system to be used
+   * @param home the home directory
+   * @return an instance of TrashPolicy
+   * @deprecated Use {@link #getInstance(Configuration, FileSystem)} instead.
+   */
+  @Deprecated
+  public static TrashPolicy getInstance(Configuration conf, FileSystem fs, Path home) {
+    Class<? extends TrashPolicy> trashClass = conf.getClass(
+        "fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
+    TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);
+    trash.initialize(conf, fs, home); // initialize TrashPolicy
+    return trash;
+  }
+
   /**
    * Get an instance of the configured TrashPolicy based on the value
    * of the configuration parameter fs.trash.classname.
@@ -107,8 +136,7 @@ public abstract class TrashPolicy extends Configured {
    * @param fs the file system to be used
    * @return an instance of TrashPolicy
    */
-  public static TrashPolicy getInstance(Configuration conf, FileSystem fs)
-      throws IOException {
+  public static TrashPolicy getInstance(Configuration conf, FileSystem fs) {
     Class<? extends TrashPolicy> trashClass = conf.getClass(
         "fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
     TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);

+ 15 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java

@@ -75,6 +75,21 @@ public class TrashPolicyDefault extends TrashPolicy {
     initialize(conf, fs);
   }
 
+  /**
+   * @deprecated Use {@link #initialize(Configuration, FileSystem)} instead.
+   */
+  @Override
+  @Deprecated
+  public void initialize(Configuration conf, FileSystem fs, Path home) {
+    this.fs = fs;
+    this.deletionInterval = (long)(conf.getFloat(
+        FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT)
+        * MSECS_PER_MINUTE);
+    this.emptierInterval = (long)(conf.getFloat(
+        FS_TRASH_CHECKPOINT_INTERVAL_KEY, FS_TRASH_CHECKPOINT_INTERVAL_DEFAULT)
+        * MSECS_PER_MINUTE);
+   }
+
   @Override
   public void initialize(Configuration conf, FileSystem fs) {
     this.fs = fs;

+ 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());
       }
     }
   }

+ 16 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java

@@ -60,20 +60,22 @@ public class SnappyCodec implements Configurable, CompressionCodec, DirectDecomp
    * Are the native snappy libraries loaded & initialized?
    */
   public static void checkNativeCodeLoaded() {
-      if (!NativeCodeLoader.isNativeCodeLoaded() ||
-          !NativeCodeLoader.buildSupportsSnappy()) {
-        throw new RuntimeException("native snappy library not available: " +
-            "this version of libhadoop was built without " +
-            "snappy support.");
-      }
-      if (!SnappyCompressor.isNativeCodeLoaded()) {
-        throw new RuntimeException("native snappy library not available: " +
-            "SnappyCompressor has not been loaded.");
-      }
-      if (!SnappyDecompressor.isNativeCodeLoaded()) {
-        throw new RuntimeException("native snappy library not available: " +
-            "SnappyDecompressor has not been loaded.");
-      }
+    if (!NativeCodeLoader.buildSupportsSnappy()) {
+      throw new RuntimeException("native snappy library not available: " +
+          "this version of libhadoop was built without " +
+          "snappy support.");
+    }
+    if (!NativeCodeLoader.isNativeCodeLoaded()) {
+      throw new RuntimeException("Failed to load libhadoop.");
+    }
+    if (!SnappyCompressor.isNativeCodeLoaded()) {
+      throw new RuntimeException("native snappy library not available: " +
+          "SnappyCompressor has not been loaded.");
+    }
+    if (!SnappyDecompressor.isNativeCodeLoaded()) {
+      throw new RuntimeException("native snappy library not available: " +
+          "SnappyDecompressor has not been loaded.");
+    }
   }
   
   public static boolean isNativeCodeLoaded() {

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

@@ -0,0 +1,88 @@
+/**
+ * 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.ExecutionException;
+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 InterruptedException, ExecutionException {
+    waitForCompletion();
+    if (error != null) {
+      throw new ExecutionException(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();
+    }
+  }
+}

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

@@ -384,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() {
@@ -672,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);
@@ -703,6 +713,7 @@ public abstract class Server {
       return "Call#" + callId + " Retry#" + retryCount;
     }
 
+    @Override
     public Void run() throws Exception {
       return null;
     }
@@ -718,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
@@ -799,6 +814,11 @@ public abstract class Server {
       this.rpcRequest = param;
     }
 
+    @Override
+    public String getProtocol() {
+      return "rpc";
+    }
+
     @Override
     public UserGroupInformation getRemoteUser() {
       return connection.user;
@@ -2333,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
@@ -2487,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) {

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

@@ -46,6 +46,7 @@ import org.apache.htrace.core.Tracer;
 
 /** An RpcEngine implementation for Writable data. */
 @InterfaceStability.Evolving
+@Deprecated
 public class WritableRpcEngine implements RpcEngine {
   private static final Log LOG = LogFactory.getLog(RPC.class);
   
@@ -331,6 +332,7 @@ public class WritableRpcEngine implements RpcEngine {
 
 
   /** An RPC Server. */
+  @Deprecated
   public static class Server extends RPC.Server {
     /** 
      * Construct an RPC server.
@@ -443,7 +445,8 @@ public class WritableRpcEngine implements RpcEngine {
         value = value.substring(0, 55)+"...";
       LOG.info(value);
     }
-    
+
+    @Deprecated
     static class WritableRpcInvoker implements RpcInvoker {
 
      @Override

+ 6 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java

@@ -47,15 +47,17 @@ import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
 import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 
 /**
  * Change log level in runtime.
  */
 @InterfaceStability.Evolving
 public class LogLevel {
-  public static final String USAGES = "\nUsage: General options are:\n"
+  public static final String USAGES = "\nUsage: Command options are:\n"
       + "\t[-getlevel <host:port> <classname> [-protocol (http|https)]\n"
       + "\t[-setlevel <host:port> <classname> <level> "
       + "[-protocol (http|https)]\n";
@@ -67,7 +69,7 @@ public class LogLevel {
    */
   public static void main(String[] args) throws Exception {
     CLI cli = new CLI(new Configuration());
-    System.exit(cli.run(args));
+    System.exit(ToolRunner.run(cli, args));
   }
 
   /**
@@ -81,6 +83,7 @@ public class LogLevel {
 
   private static void printUsage() {
     System.err.println(USAGES);
+    GenericOptionsParser.printGenericCommandUsage(System.err);
   }
 
   public static boolean isValidProtocol(String protocol) {
@@ -107,7 +110,7 @@ public class LogLevel {
         sendLogLevelRequest();
       } catch (HadoopIllegalArgumentException e) {
         printUsage();
-        throw e;
+        return -1;
       }
       return 0;
     }

+ 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;
     }

+ 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() {
+  }
+}

+ 96 - 107
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);
@@ -968,67 +946,68 @@ public class UserGroupInformation {
 
   /**Spawn a thread to do periodic renewals of kerberos credentials*/
   private void spawnAutoRenewalThreadForUserCreds() {
-    if (isSecurityEnabled()) {
-      //spawn thread only if we have kerb credentials
-      if (user.getAuthenticationMethod() == AuthenticationMethod.KERBEROS &&
-          !isKeytab) {
-        Thread t = new Thread(new Runnable() {
-          
-          @Override
-          public void run() {
-            String cmd = conf.get("hadoop.kerberos.kinit.command",
-                                  "kinit");
-            KerberosTicket tgt = getTGT();
+    if (!isSecurityEnabled()
+        || user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS
+        || isKeytab) {
+      return;
+    }
+
+    //spawn thread only if we have kerb credentials
+    Thread t = new Thread(new Runnable() {
+
+      @Override
+      public void run() {
+        String cmd = conf.get("hadoop.kerberos.kinit.command", "kinit");
+        KerberosTicket tgt = getTGT();
+        if (tgt == null) {
+          return;
+        }
+        long nextRefresh = getRefreshTime(tgt);
+        while (true) {
+          try {
+            long now = Time.now();
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Current time is " + now);
+              LOG.debug("Next refresh is " + nextRefresh);
+            }
+            if (now < nextRefresh) {
+              Thread.sleep(nextRefresh - now);
+            }
+            Shell.execCommand(cmd, "-R");
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("renewed ticket");
+            }
+            reloginFromTicketCache();
+            tgt = getTGT();
             if (tgt == null) {
+              LOG.warn("No TGT after renewal. Aborting renew thread for " +
+                  getUserName());
               return;
             }
-            long nextRefresh = getRefreshTime(tgt);
-            while (true) {
-              try {
-                long now = Time.now();
-                if(LOG.isDebugEnabled()) {
-                  LOG.debug("Current time is " + now);
-                  LOG.debug("Next refresh is " + nextRefresh);
-                }
-                if (now < nextRefresh) {
-                  Thread.sleep(nextRefresh - now);
-                }
-                Shell.execCommand(cmd, "-R");
-                if(LOG.isDebugEnabled()) {
-                  LOG.debug("renewed ticket");
-                }
-                reloginFromTicketCache();
-                tgt = getTGT();
-                if (tgt == null) {
-                  LOG.warn("No TGT after renewal. Aborting renew thread for " +
-                           getUserName());
-                  return;
-                }
-                nextRefresh = Math.max(getRefreshTime(tgt),
-                                       now + kerberosMinSecondsBeforeRelogin);
-              } catch (InterruptedException ie) {
-                LOG.warn("Terminating renewal thread");
-                return;
-              } catch (IOException ie) {
-                LOG.warn("Exception encountered while running the" +
-                    " renewal command. Aborting renew thread. " + ie);
-                return;
-              }
-            }
+            nextRefresh = Math.max(getRefreshTime(tgt),
+              now + kerberosMinSecondsBeforeRelogin);
+          } catch (InterruptedException ie) {
+            LOG.warn("Terminating renewal thread");
+            return;
+          } catch (IOException ie) {
+            LOG.warn("Exception encountered while running the" +
+                " renewal command. Aborting renew thread. " + ie);
+            return;
           }
-        });
-        t.setDaemon(true);
-        t.setName("TGT Renewer for " + getUserName());
-        t.start();
+        }
       }
-    }
+    });
+    t.setDaemon(true);
+    t.setName("TGT Renewer for " + getUserName());
+    t.start();
   }
   /**
    * Log a user in from a keytab file. Loads a user identity from a keytab
    * 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

+ 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.

+ 3 - 1
hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md

@@ -202,7 +202,9 @@ Manage keys via the KeyProvider. For details on KeyProviders, see the [Transpare
 
 Providers frequently require that a password or other secret is supplied. If the provider requires a password and is unable to find one, it will use a default password and emit a warning message that the default password is being used. If the `-strict` flag is supplied, the warning message becomes an error message and the command returns immediately with an error status.
 
-NOTE: Some KeyProviders (e.g. org.apache.hadoop.crypto.key.JavaKeyStoreProvider) does not support uppercase key names.
+NOTE: Some KeyProviders (e.g. org.apache.hadoop.crypto.key.JavaKeyStoreProvider) do not support uppercase key names.
+
+NOTE: Some KeyProviders do not directly execute a key deletion (e.g. performs a soft-delete instead, or delay the actual deletion, to prevent mistake). In these cases, one may encounter errors when creating/deleting a key with the same name after deleting it. Please check the underlying KeyProvider for details.
 
 ### `trace`
 

+ 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();

+ 140 - 24
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;
@@ -169,6 +168,9 @@ public class TestConfiguration extends TestCase {
     declareProperty("my.fullfile", "${my.base}/${my.file}${my.suffix}", "/tmp/hadoop_user/hello.txt");
     // check that undefined variables are returned as-is
     declareProperty("my.failsexpand", "a${my.undefvar}b", "a${my.undefvar}b");
+    // check that multiple variable references are resolved
+    declareProperty("my.user.group", "${user.name} ${user.name}",
+        "hadoop_user hadoop_user");
     endConfig();
     Path fileResource = new Path(CONFIG);
     mock.addResource(fileResource);
@@ -1140,7 +1142,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 +1185,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 +1201,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();
@@ -1376,7 +1511,7 @@ public class TestConfiguration extends TestCase {
     }
   }
 
-  public void testInvalidSubstitutation() {
+  public void testInvalidSubstitution() {
     final Configuration configuration = new Configuration(false);
 
     // 2-var loops
@@ -1390,25 +1525,6 @@ public class TestConfiguration extends TestCase {
       configuration.set(key, keyExpression);
       assertEquals("Unexpected value", keyExpression, configuration.get(key));
     }
-
-    //
-    // 3-variable loops
-    //
-
-    final String expVal1 = "${test.var2}";
-    String testVar1 = "test.var1";
-    configuration.set(testVar1, expVal1);
-    configuration.set("test.var2", "${test.var3}");
-    configuration.set("test.var3", "${test.var1}");
-    assertEquals("Unexpected value", expVal1, configuration.get(testVar1));
-
-    // 3-variable loop with non-empty value prefix/suffix
-    //
-    final String expVal2 = "foo2${test.var2}bar2";
-    configuration.set(testVar1, expVal2);
-    configuration.set("test.var2", "foo3${test.var3}bar3");
-    configuration.set("test.var3", "foo1${test.var1}bar1");
-    assertEquals("Unexpected value", expVal2, configuration.get(testVar1));
   }
 
   public void testIncompleteSubbing() {

+ 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;
+  }
+
+}

+ 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);
   }
 }

+ 4 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java

@@ -691,6 +691,10 @@ public class TestTrash extends TestCase {
   public static class TestTrashPolicy extends TrashPolicy {
     public TestTrashPolicy() { }
 
+    @Override
+    public void initialize(Configuration conf, FileSystem fs, Path home) {
+    }
+
     @Override
     public void initialize(Configuration conf, FileSystem fs) {
     }

+ 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);
     }

+ 39 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -393,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()) {

+ 21 - 13
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java

@@ -21,6 +21,7 @@ import static org.junit.Assert.*;
 
 import java.security.NoSuchAlgorithmException;
 
+import com.google.common.base.Supplier;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -441,12 +442,16 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
     cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
     cluster.waitForActiveLockHolder(0);
 
-    Thread.sleep(10000); // allow to quiesce
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return cluster.getService(0).fenceCount == 0 &&
+            cluster.getService(1).fenceCount == 0 &&
+            cluster.getService(0).activeTransitionCount == 2 &&
+            cluster.getService(1).activeTransitionCount == 1;
+      }
+    }, 100, 60 * 1000);
 
-    assertEquals(0, cluster.getService(0).fenceCount);
-    assertEquals(0, cluster.getService(1).fenceCount);
-    assertEquals(2, cluster.getService(0).activeTransitionCount);
-    assertEquals(1, cluster.getService(1).activeTransitionCount);
   }
 
   @Test
@@ -590,14 +595,17 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
     cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
     cluster.waitForActiveLockHolder(0);
 
-    Thread.sleep(10000); // allow to quiesce
-
-    assertEquals(0, cluster.getService(0).fenceCount);
-    assertEquals(0, cluster.getService(1).fenceCount);
-    assertEquals(0, cluster.getService(2).fenceCount);
-    assertEquals(2, cluster.getService(0).activeTransitionCount);
-    assertEquals(1, cluster.getService(1).activeTransitionCount);
-    assertEquals(1, cluster.getService(2).activeTransitionCount);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return cluster.getService(0).fenceCount == 0 &&
+            cluster.getService(1).fenceCount == 0 &&
+            cluster.getService(2).fenceCount == 0 &&
+            cluster.getService(0).activeTransitionCount == 2 &&
+            cluster.getService(1).activeTransitionCount == 1 &&
+            cluster.getService(2).activeTransitionCount == 1;
+      }
+    }, 100, 60 * 1000);
   }
 
   private int runFC(DummyHAService target, String ... args) throws Exception {

+ 87 - 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;
@@ -71,6 +72,7 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -926,6 +928,91 @@ 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 (ExecutionException ee) {
+        assertTrue((ee.getCause()) instanceof IOException);
+        assertEquals(expectedIOE.getMessage(), ee.getCause().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;

+ 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

+ 397 - 268
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,83 +104,101 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   @SuppressWarnings("unchecked")
   public Response createKey(Map jsonKey) throws Exception {
-    KMSWebApp.getAdminCallsMeter().mark();
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
-    KMSClientProvider.checkNotEmpty(name, KMSRESTConstants.NAME_FIELD);
-    assertAccess(KMSACLs.Type.CREATE, user, KMSOp.CREATE_KEY, name);
-    String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
-    final String material = (String) jsonKey.get(KMSRESTConstants.MATERIAL_FIELD);
-    int length = (jsonKey.containsKey(KMSRESTConstants.LENGTH_FIELD))
-                 ? (Integer) jsonKey.get(KMSRESTConstants.LENGTH_FIELD) : 0;
-    String description = (String)
-        jsonKey.get(KMSRESTConstants.DESCRIPTION_FIELD);
-    Map<String, String> attributes = (Map<String, String>)
-        jsonKey.get(KMSRESTConstants.ATTRIBUTES_FIELD);
-    if (material != null) {
-      assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
-          KMSOp.CREATE_KEY, name);
-    }
-    final KeyProvider.Options options = new KeyProvider.Options(
-        KMSWebApp.getConfiguration());
-    if (cipher != null) {
-      options.setCipher(cipher);
-    }
-    if (length != 0) {
-      options.setBitLength(length);
-    }
-    options.setDescription(description);
-    options.setAttributes(attributes);
-
-    KeyProvider.KeyVersion keyVersion = user.doAs(
-        new PrivilegedExceptionAction<KeyVersion>() {
-          @Override
-          public KeyVersion run() throws Exception {
-            KeyProvider.KeyVersion keyVersion = (material != null)
-              ? provider.createKey(name, Base64.decodeBase64(material), options)
-              : provider.createKey(name, options);
-            provider.flush();
-            return keyVersion;
+    try{
+      LOG.trace("Entering createKey Method.");
+      KMSWebApp.getAdminCallsMeter().mark();
+      UserGroupInformation user = HttpUserGroupInformation.get();
+      final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
+      KMSClientProvider.checkNotEmpty(name, KMSRESTConstants.NAME_FIELD);
+      assertAccess(KMSACLs.Type.CREATE, user, KMSOp.CREATE_KEY, name);
+      String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
+      final String material;
+      material = (String) jsonKey.get(KMSRESTConstants.MATERIAL_FIELD);
+      int length = (jsonKey.containsKey(KMSRESTConstants.LENGTH_FIELD))
+                   ? (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) {
+        assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
+            KMSOp.CREATE_KEY, name);
+      }
+      final KeyProvider.Options options = new KeyProvider.Options(
+          KMSWebApp.getConfiguration());
+      if (cipher != null) {
+        options.setCipher(cipher);
+      }
+      if (length != 0) {
+        options.setBitLength(length);
+      }
+      options.setDescription(description);
+      options.setAttributes(attributes);
+
+      KeyProvider.KeyVersion keyVersion = user.doAs(
+          new PrivilegedExceptionAction<KeyVersion>() {
+            @Override
+            public KeyVersion run() throws Exception {
+              KeyProvider.KeyVersion keyVersion = (material != null)
+                  ? provider.createKey(name, Base64.decodeBase64(material),
+                      options)
+                  : provider.createKey(name, options);
+              provider.flush();
+              return keyVersion;
+            }
           }
-        }
-    );
+      );
 
-    kmsAudit.ok(user, KMSOp.CREATE_KEY, name, "UserProvidedMaterial:" +
-        (material != null) + " Description:" + description);
+      kmsAudit.ok(user, KMSOp.CREATE_KEY, name, "UserProvidedMaterial:" +
+          (material != null) + " Description:" + description);
 
-    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
-      keyVersion = removeKeyMaterial(keyVersion);
+      if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
+        keyVersion = removeKeyMaterial(keyVersion);
+      }
+      Map json = KMSServerJSONUtils.toJSON(keyVersion);
+      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();
+    } catch (Exception e) {
+      LOG.debug("Exception in createKey.", e);
+      throw e;
     }
-    Map json = KMSServerJSONUtils.toJSON(keyVersion);
-    String requestURL = KMSMDCFilter.getURL();
-    int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE);
-    requestURL = requestURL.substring(0, idx);
-    return Response.created(getKeyURI(KMSRESTConstants.SERVICE_VERSION, name))
-        .type(MediaType.APPLICATION_JSON)
-        .header("Location", getKeyURI(requestURL, name)).entity(json).build();
   }
 
   @DELETE
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   public Response deleteKey(@PathParam("name") final String name)
       throws Exception {
-    KMSWebApp.getAdminCallsMeter().mark();
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
-    KMSClientProvider.checkNotEmpty(name, "name");
-
-    user.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-        provider.deleteKey(name);
-        provider.flush();
-        return null;
-      }
-    });
-
-    kmsAudit.ok(user, KMSOp.DELETE_KEY, name, "");
-
-    return Response.ok().build();
+    try {
+      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 {
+          provider.deleteKey(name);
+          provider.flush();
+          return null;
+        }
+      });
+
+      kmsAudit.ok(user, KMSOp.DELETE_KEY, name, "");
+      LOG.trace("Exiting deleteKey method.");
+      return Response.ok().build();
+    } catch (Exception e) {
+      LOG.debug("Exception in deleteKey.", e);
+      throw e;
+    }
   }
 
   @POST
@@ -185,38 +207,49 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response rolloverKey(@PathParam("name") final String name,
       Map jsonMaterial) throws Exception {
-    KMSWebApp.getAdminCallsMeter().mark();
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.ROLL_NEW_VERSION, name);
-    KMSClientProvider.checkNotEmpty(name, "name");
-    final String material = (String)
-        jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
-    if (material != null) {
-      assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
-          KMSOp.ROLL_NEW_VERSION, name);
-    }
+    try {
+      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) {
+        assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
+                KMSOp.ROLL_NEW_VERSION, name);
+      }
 
-    KeyProvider.KeyVersion keyVersion = user.doAs(
-        new PrivilegedExceptionAction<KeyVersion>() {
-          @Override
-          public KeyVersion run() throws Exception {
-            KeyVersion keyVersion = (material != null)
-              ? provider.rollNewVersion(name, Base64.decodeBase64(material))
-              : provider.rollNewVersion(name);
-            provider.flush();
-            return keyVersion;
-          }
-        }
-    );
+      KeyProvider.KeyVersion keyVersion = user.doAs(
+              new PrivilegedExceptionAction<KeyVersion>() {
+              @Override
+                public KeyVersion run() throws Exception {
+                KeyVersion keyVersion = (material != null)
+                        ? provider.rollNewVersion(name,
+                        Base64.decodeBase64(material))
+                        : provider.rollNewVersion(name);
+                provider.flush();
+                return keyVersion;
+              }
+            }
+      );
 
-    kmsAudit.ok(user, KMSOp.ROLL_NEW_VERSION, name, "UserProvidedMaterial:" +
-        (material != null) + " NewVersion:" + keyVersion.getVersionName());
+      kmsAudit.ok(user, KMSOp.ROLL_NEW_VERSION, name, "UserProvidedMaterial:" +
+              (material != null) +
+              " NewVersion:" + keyVersion.getVersionName());
 
-    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
-      keyVersion = removeKeyMaterial(keyVersion);
+      if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
+        keyVersion = removeKeyMaterial(keyVersion);
+      }
+      Map json = KMSServerJSONUtils.toJSON(keyVersion);
+      LOG.trace("Exiting rolloverKey Method.");
+      return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
+              .build();
+    } catch (Exception e) {
+      LOG.debug("Exception in rolloverKey.", e);
+      throw e;
     }
-    Map json = KMSServerJSONUtils.toJSON(keyVersion);
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
   @GET
@@ -224,52 +257,76 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeysMetadata(@QueryParam(KMSRESTConstants.KEY)
       List<String> keyNamesList) throws Exception {
-    KMSWebApp.getAdminCallsMeter().mark();
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    final String[] keyNames = keyNamesList.toArray(
-        new String[keyNamesList.size()]);
-    assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_KEYS_METADATA);
-
-    KeyProvider.Metadata[] keysMeta = user.doAs(
-        new PrivilegedExceptionAction<KeyProvider.Metadata[]>() {
-          @Override
-          public KeyProvider.Metadata[] run() throws Exception {
-            return provider.getKeysMetadata(keyNames);
-          }
-        }
-    );
+    try {
+      LOG.trace("Entering getKeysMetadata method.");
+      KMSWebApp.getAdminCallsMeter().mark();
+      UserGroupInformation user = HttpUserGroupInformation.get();
+      final String[] keyNames = keyNamesList.toArray(
+              new String[keyNamesList.size()]);
+      assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_KEYS_METADATA);
+
+      KeyProvider.Metadata[] keysMeta = user.doAs(
+              new PrivilegedExceptionAction<KeyProvider.Metadata[]>() {
+              @Override
+                public KeyProvider.Metadata[] run() throws Exception {
+                return provider.getKeysMetadata(keyNames);
+              }
+            }
+      );
 
-    Object json = KMSServerJSONUtils.toJSON(keyNames, keysMeta);
-    kmsAudit.ok(user, KMSOp.GET_KEYS_METADATA, "");
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+      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();
+    } catch (Exception e) {
+      LOG.debug("Exception in getKeysmetadata.", e);
+      throw e;
+    }
   }
 
   @GET
   @Path(KMSRESTConstants.KEYS_NAMES_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyNames() throws Exception {
-    KMSWebApp.getAdminCallsMeter().mark();
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    assertAccess(KMSACLs.Type.GET_KEYS, user, KMSOp.GET_KEYS);
-
-    List<String> json = user.doAs(
-        new PrivilegedExceptionAction<List<String>>() {
-          @Override
-          public List<String> run() throws Exception {
-            return provider.getKeys();
-          }
-        }
-    );
+    try {
+      LOG.trace("Entering getKeyNames method.");
+      KMSWebApp.getAdminCallsMeter().mark();
+      UserGroupInformation user = HttpUserGroupInformation.get();
+      assertAccess(KMSACLs.Type.GET_KEYS, user, KMSOp.GET_KEYS);
+
+      List<String> json = user.doAs(
+              new PrivilegedExceptionAction<List<String>>() {
+              @Override
+                public List<String> run() throws Exception {
+                return provider.getKeys();
+              }
+            }
+      );
 
-    kmsAudit.ok(user, KMSOp.GET_KEYS, "");
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+      kmsAudit.ok(user, KMSOp.GET_KEYS, "");
+      LOG.trace("Exiting getKeyNames method.");
+      return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
+              .build();
+    } catch (Exception e) {
+      LOG.debug("Exception in getkeyNames.", e);
+      throw e;
+    }
   }
 
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   public Response getKey(@PathParam("name") String name)
       throws Exception {
-    return getMetadata(name);
+    try {
+      LOG.trace("Entering getKey method.");
+      LOG.debug("Getting key information for key with name {}.", name);
+      LOG.trace("Exiting getKey method.");
+      return getMetadata(name);
+    } catch (Exception e) {
+      LOG.debug("Exception in getKey.", e);
+      throw e;
+    }
   }
 
   @GET
@@ -278,23 +335,32 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getMetadata(@PathParam("name") final String name)
       throws Exception {
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    KMSClientProvider.checkNotEmpty(name, "name");
-    KMSWebApp.getAdminCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
-
-    KeyProvider.Metadata metadata = user.doAs(
-        new PrivilegedExceptionAction<KeyProvider.Metadata>() {
-          @Override
-          public KeyProvider.Metadata run() throws Exception {
-            return provider.getMetadata(name);
-          }
-        }
-    );
+    try {
+      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>() {
+              @Override
+                public KeyProvider.Metadata run() throws Exception {
+                return provider.getMetadata(name);
+              }
+            }
+      );
 
-    Object json = KMSServerJSONUtils.toJSON(name, metadata);
-    kmsAudit.ok(user, KMSOp.GET_METADATA, name, "");
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+      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();
+    } catch (Exception e) {
+      LOG.debug("Exception in getMetadata.", e);
+      throw e;
+    }
   }
 
   @GET
@@ -303,23 +369,32 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getCurrentVersion(@PathParam("name") final String name)
       throws Exception {
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    KMSClientProvider.checkNotEmpty(name, "name");
-    KMSWebApp.getKeyCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
-
-    KeyVersion keyVersion = user.doAs(
-        new PrivilegedExceptionAction<KeyVersion>() {
-          @Override
-          public KeyVersion run() throws Exception {
-            return provider.getCurrentKey(name);
-          }
-        }
-    );
+    try {
+      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>() {
+              @Override
+                public KeyVersion run() throws Exception {
+                return provider.getCurrentKey(name);
+            }
+            }
+      );
 
-    Object json = KMSServerJSONUtils.toJSON(keyVersion);
-    kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+      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();
+    } catch (Exception e) {
+      LOG.debug("Exception in getCurrentVersion.", e);
+      throw e;
+    }
   }
 
   @GET
@@ -327,25 +402,34 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyVersion(
       @PathParam("versionName") final String versionName) throws Exception {
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    KMSClientProvider.checkNotEmpty(versionName, "versionName");
-    KMSWebApp.getKeyCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
-
-    KeyVersion keyVersion = user.doAs(
-        new PrivilegedExceptionAction<KeyVersion>() {
-          @Override
-          public KeyVersion run() throws Exception {
-            return provider.getKeyVersion(versionName);
-          }
-        }
-    );
+    try {
+      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>() {
+              @Override
+                public KeyVersion run() throws Exception {
+                return provider.getKeyVersion(versionName);
+              }
+            }
+      );
 
-    if (keyVersion != null) {
-      kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
+      if (keyVersion != null) {
+        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();
+    } catch (Exception e) {
+      LOG.debug("Exception in getKeyVersion.", e);
+      throw e;
     }
-    Object json = KMSServerJSONUtils.toJSON(keyVersion);
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
   @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -359,46 +443,65 @@ public class KMS {
           @DefaultValue("1")
           @QueryParam(KMSRESTConstants.EEK_NUM_KEYS) final int numKeys)
           throws Exception {
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    KMSClientProvider.checkNotEmpty(name, "name");
-    KMSClientProvider.checkNotNull(edekOp, "eekOp");
-
-    Object retJSON;
-    if (edekOp.equals(KMSRESTConstants.EEK_GENERATE)) {
-      assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.GENERATE_EEK, name);
-
-      final List<EncryptedKeyVersion> retEdeks =
-          new LinkedList<EncryptedKeyVersion>();
-      try {
-
-        user.doAs(
-            new PrivilegedExceptionAction<Void>() {
-              @Override
-              public Void run() throws Exception {
-                for (int i = 0; i < numKeys; i++) {
-                  retEdeks.add(provider.generateEncryptedKey(name));
+    try {
+      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 =
+                new LinkedList<EncryptedKeyVersion>();
+        try {
+
+          user.doAs(
+                  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));
+                    }
+                    return null;
+                  }
                 }
-                return null;
-              }
-            }
-        );
+          );
 
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
-      retJSON = new ArrayList();
-      for (EncryptedKeyVersion edek : retEdeks) {
-        ((ArrayList)retJSON).add(KMSServerJSONUtils.toJSON(edek));
+        } catch (Exception e) {
+          LOG.error("Exception in generateEncryptedKeys:", e);
+          throw new IOException(e);
+        }
+        kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
+        retJSON = new ArrayList();
+        for (EncryptedKeyVersion edek : retEdeks) {
+          ((ArrayList) retJSON).add(KMSServerJSONUtils.toJSON(edek));
+        }
+      } else {
+        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());
       }
-    } else {
-      throw new IllegalArgumentException("Wrong " + KMSRESTConstants.EEK_OP +
-          " value, it must be " + KMSRESTConstants.EEK_GENERATE + " or " +
-          KMSRESTConstants.EEK_DECRYPT);
+      KMSWebApp.getGenerateEEKCallsMeter().mark();
+      LOG.trace("Exiting generateEncryptedKeys method.");
+      return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
+              .build();
+    } catch (Exception e) {
+      LOG.debug("Exception in generateEncryptedKeys.", e);
+      throw e;
     }
-    KMSWebApp.getGenerateEEKCallsMeter().mark();
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
-        .build();
   }
 
   @SuppressWarnings("rawtypes")
@@ -411,47 +514,64 @@ public class KMS {
       @QueryParam(KMSRESTConstants.EEK_OP) String eekOp,
       Map jsonPayload)
       throws Exception {
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    KMSClientProvider.checkNotEmpty(versionName, "versionName");
-    KMSClientProvider.checkNotNull(eekOp, "eekOp");
-
-    final String keyName = (String) jsonPayload.get(
-        KMSRESTConstants.NAME_FIELD);
-    String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
-    String encMaterialStr = 
-        (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
-    Object retJSON;
-    if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
-      assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK, keyName);
-      KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
-      final byte[] iv = Base64.decodeBase64(ivStr);
-      KMSClientProvider.checkNotNull(encMaterialStr,
-          KMSRESTConstants.MATERIAL_FIELD);
-      final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
-
-      KeyProvider.KeyVersion retKeyVersion = user.doAs(
-          new PrivilegedExceptionAction<KeyVersion>() {
-            @Override
-            public KeyVersion run() throws Exception {
-              return provider.decryptEncryptedKey(
-                  new KMSClientProvider.KMSEncryptedKeyVersion(keyName,
-                      versionName, iv, KeyProviderCryptoExtension.EEK,
-                      encMaterial)
-              );
-            }
-          }
-      );
+    try {
+      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) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
+      Object retJSON;
+      if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
+        assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK,
+                keyName);
+        KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
+        final byte[] iv = Base64.decodeBase64(ivStr);
+        KMSClientProvider.checkNotNull(encMaterialStr,
+                KMSRESTConstants.MATERIAL_FIELD);
+        final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
+
+        KeyProvider.KeyVersion retKeyVersion = user.doAs(
+                new PrivilegedExceptionAction<KeyVersion>() {
+                @Override
+                  public KeyVersion run() throws Exception {
+                  return provider.decryptEncryptedKey(
+                            new KMSClientProvider.KMSEncryptedKeyVersion(
+                                    keyName, versionName, iv,
+                                            KeyProviderCryptoExtension.EEK,
+                                            encMaterial)
+                    );
+                }
+              }
+        );
 
-      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);
+        retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
+        kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
+      } else {
+        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();
+    } catch (Exception e) {
+      LOG.debug("Exception in decryptEncryptedKey.", e);
+      throw e;
     }
-    KMSWebApp.getDecryptEEKCallsMeter().mark();
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
-        .build();
   }
 
   @GET
@@ -460,23 +580,32 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyVersions(@PathParam("name") final String name)
       throws Exception {
-    UserGroupInformation user = HttpUserGroupInformation.get();
-    KMSClientProvider.checkNotEmpty(name, "name");
-    KMSWebApp.getKeyCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
-
-    List<KeyVersion> ret = user.doAs(
-        new PrivilegedExceptionAction<List<KeyVersion>>() {
-          @Override
-          public List<KeyVersion> run() throws Exception {
-            return provider.getKeyVersions(name);
-          }
-        }
-    );
+    try {
+      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>>() {
+              @Override
+                public List<KeyVersion> run() throws Exception {
+                return provider.getKeyVersions(name);
+              }
+            }
+      );
 
-    Object json = KMSServerJSONUtils.toJSON(ret);
-    kmsAudit.ok(user, KMSOp.GET_KEY_VERSIONS, name, "");
-    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+      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();
+    } catch (Exception e) {
+      LOG.debug("Exception in getKeyVersions.", e);
+      throw e;
+    }
   }
 
 }

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

@@ -2599,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);
     }
   }
 

+ 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 {

+ 17 - 4
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;
@@ -328,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);
   }
 
@@ -354,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.

+ 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>

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -48,6 +48,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.HTTPFS_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.HTTP_BUFFER_SIZE_DEFAULT;
+
 /**
  * FileSystem operation executors used by {@link HttpFSServer}.
  */
@@ -462,7 +465,8 @@ public class FSOperations {
         blockSize = fs.getDefaultBlockSize(path);
       }
       FsPermission fsPermission = new FsPermission(permission);
-      int bufferSize = fs.getConf().getInt("httpfs.buffer.size", 4096);
+      int bufferSize = fs.getConf().getInt(HTTPFS_BUFFER_SIZE_KEY,
+          HTTP_BUFFER_SIZE_DEFAULT);
       OutputStream os = fs.create(path, fsPermission, override, bufferSize, replication, blockSize, null);
       IOUtils.copyBytes(is, os, bufferSize, true);
       os.close();
@@ -752,7 +756,8 @@ public class FSOperations {
      */
     @Override
     public InputStream execute(FileSystem fs) throws IOException {
-      int bufferSize = HttpFSServerWebApp.get().getConfig().getInt("httpfs.buffer.size", 4096);
+      int bufferSize = HttpFSServerWebApp.get().getConfig().getInt(
+          HTTPFS_BUFFER_SIZE_KEY, HTTP_BUFFER_SIZE_DEFAULT);
       return fs.open(path, bufferSize);
     }
 

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java

@@ -50,6 +50,8 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+
 @InterfaceAudience.Private
 public class FileSystemAccessService extends BaseService implements FileSystemAccess {
   private static final Logger LOG = LoggerFactory.getLogger(FileSystemAccessService.class);
@@ -159,7 +161,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
         throw new ServiceException(FileSystemAccessException.ERROR.H01, KERBEROS_PRINCIPAL);
       }
       Configuration conf = new Configuration();
-      conf.set("hadoop.security.authentication", "kerberos");
+      conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
       UserGroupInformation.setConfiguration(conf);
       try {
         UserGroupInformation.loginUserFromKeytab(principal, keytab);
@@ -169,7 +171,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
       LOG.info("Using FileSystemAccess Kerberos authentication, principal [{}] keytab [{}]", principal, keytab);
     } else if (security.equals("simple")) {
       Configuration conf = new Configuration();
-      conf.set("hadoop.security.authentication", "simple");
+      conf.set(HADOOP_SECURITY_AUTHENTICATION, "simple");
       UserGroupInformation.setConfiguration(conf);
       LOG.info("Using FileSystemAccess simple/pseudo authentication, principal [{}]", System.getProperty("user.name"));
     } else {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h

@@ -493,6 +493,7 @@ extern  "C" {
      *         complete before proceeding with further file updates.
      *         -1 on error.
      */
+    LIBHDFS_EXTERNAL
     int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength);
 
     /**

+ 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

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

@@ -70,6 +70,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.webhdfs.ugi.expire.after.access";
   public static final int     DFS_WEBHDFS_UGI_EXPIRE_AFTER_ACCESS_DEFAULT =
       10*60*1000; //10 minutes
+  public static final String DFS_WEBHDFS_USE_IPC_CALLQ =
+      "dfs.webhdfs.use.ipc.callq";
+  public static final boolean DFS_WEBHDFS_USE_IPC_CALLQ_DEFAULT = true;
 
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";
@@ -992,6 +995,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.disk.balancer.plan.threshold.percent";
   public static final int DFS_DISK_BALANCER_PLAN_THRESHOLD_DEFAULT = 10;
 
+  public static final String HTTPFS_BUFFER_SIZE_KEY =
+      "httpfs.buffer.size";
+  public static final int HTTP_BUFFER_SIZE_DEFAULT = 4096;
 
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated

+ 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)

+ 29 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -103,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;
 
@@ -1059,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 "
@@ -1068,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;
@@ -1145,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;
     }
   }
 
@@ -1766,8 +1781,12 @@ public class BlockManager implements BlockStatsMXBean {
 
   private boolean isInNewRack(DatanodeDescriptor[] srcs,
       DatanodeDescriptor target) {
+    LOG.debug("check if target {} increases racks, srcs={}", target,
+        Arrays.asList(srcs));
     for (DatanodeDescriptor src : srcs) {
-      if (src.getNetworkLocation().equals(target.getNetworkLocation())) {
+      if (!src.isDecommissionInProgress() &&
+          src.getNetworkLocation().equals(target.getNetworkLocation())) {
+        LOG.debug("the target {} is in the same rack with src {}", target, src);
         return false;
       }
     }
@@ -4005,13 +4024,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>() {

+ 28 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java

@@ -388,6 +388,10 @@ public class DecommissionManager {
      * The number of blocks that have been checked on this tick.
      */
     private int numBlocksChecked = 0;
+    /**
+     * The number of blocks checked after (re)holding lock.
+     */
+    private int numBlocksCheckedPerLock = 0;
     /**
      * The number of nodes that have been checked on this tick. Used for 
      * statistics.
@@ -418,6 +422,7 @@ public class DecommissionManager {
       }
       // Reset the checked count at beginning of each iteration
       numBlocksChecked = 0;
+      numBlocksCheckedPerLock = 0;
       numNodesChecked = 0;
       // Check decom progress
       namesystem.writeLock();
@@ -451,7 +456,8 @@ public class DecommissionManager {
               iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
-      while (it.hasNext() && !exceededNumBlocksPerCheck()) {
+      while (it.hasNext() && !exceededNumBlocksPerCheck() && namesystem
+          .isRunning()) {
         numNodesChecked++;
         final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
             entry = it.next();
@@ -577,7 +583,28 @@ public class DecommissionManager {
       int decommissionOnlyReplicas = 0;
       int lowRedundancyInOpenFiles = 0;
       while (it.hasNext()) {
+        if (insufficientList == null
+            && numBlocksCheckedPerLock >= numBlocksPerCheck) {
+          // During fullscan insufficientlyReplicated will NOT be null, iterator
+          // will be DN's iterator. So should not yield lock, otherwise
+          // ConcurrentModificationException could occur.
+          // Once the fullscan done, iterator will be a copy. So can yield the
+          // lock.
+          // Yielding is required in case of block number is greater than the
+          // configured per-iteration-limit.
+          namesystem.writeUnlock();
+          try {
+            LOG.debug("Yielded lock during decommission check");
+            Thread.sleep(0, 500);
+          } catch (InterruptedException ignored) {
+            return;
+          }
+          // reset
+          numBlocksCheckedPerLock = 0;
+          namesystem.writeLock();
+        }
         numBlocksChecked++;
+        numBlocksCheckedPerLock++;
         final BlockInfo block = it.next();
         // Remove the block from the list if it's no longer in the block map,
         // e.g. the containing file has been deleted

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -370,7 +370,7 @@ public interface HdfsServerConstants {
   String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
   String XATTR_ERASURECODING_POLICY =
-      "raw.hdfs.erasurecoding.policy";
+      "system.hdfs.erasurecoding.policy";
 
   long BLOCK_GROUP_INDEX_MASK = 15;
   byte MAX_BLOCKS_IN_GROUP = 16;

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.apache.hadoop.util.ToolRunner;
@@ -269,11 +270,17 @@ public abstract class Storage extends StorageInfo {
 
     private String storageUuid = null;      // Storage directory identifier.
     
+    private final StorageLocation location;
     public StorageDirectory(File dir) {
       // default dirType is null
       this(dir, null, false);
     }
     
+    public StorageDirectory(StorageLocation location) {
+      // default dirType is null
+      this(location.getFile(), null, false, location);
+    }
+
     public StorageDirectory(File dir, StorageDirType dirType) {
       this(dir, dirType, false);
     }
@@ -294,11 +301,22 @@ public abstract class Storage extends StorageInfo {
      *          disables locking on the storage directory, false enables locking
      */
     public StorageDirectory(File dir, StorageDirType dirType, boolean isShared) {
+      this(dir, dirType, isShared, null);
+    }
+
+    public StorageDirectory(File dir, StorageDirType dirType,
+        boolean isShared, StorageLocation location) {
       this.root = dir;
       this.lock = null;
       this.dirType = dirType;
       this.isShared = isShared;
+      this.location = location;
+      assert location == null ||
+          dir.getAbsolutePath().startsWith(
+              location.getFile().getAbsolutePath()):
+            "The storage location and directory should be equal";
     }
+
     
     /**
      * Get root directory of this storage
@@ -861,6 +879,10 @@ public abstract class Storage extends StorageInfo {
       }
       return false;
     }
+
+    public StorageLocation getStorageLocation() {
+      return location;
+    }
   }
 
   /**

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -147,10 +147,10 @@ public class BlockPoolSliceStorage extends Storage {
    * @throws IOException
    */
   private StorageDirectory loadStorageDirectory(NamespaceInfo nsInfo,
-      File dataDir, StartupOption startOpt,
+      File dataDir, StorageLocation location, StartupOption startOpt,
       List<Callable<StorageDirectory>> callables, Configuration conf)
           throws IOException {
-    StorageDirectory sd = new StorageDirectory(dataDir, null, true);
+    StorageDirectory sd = new StorageDirectory(dataDir, null, true, location);
     try {
       StorageState curState = sd.analyzeStorage(startOpt, this, true);
       // sd is locked but not opened
@@ -208,9 +208,9 @@ public class BlockPoolSliceStorage extends Storage {
    * @throws IOException on error
    */
   List<StorageDirectory> loadBpStorageDirectories(NamespaceInfo nsInfo,
-      Collection<File> dataDirs, StartupOption startOpt,
-      List<Callable<StorageDirectory>> callables, Configuration conf)
-          throws IOException {
+      Collection<File> dataDirs, StorageLocation location,
+      StartupOption startOpt, List<Callable<StorageDirectory>> callables,
+      Configuration conf) throws IOException {
     List<StorageDirectory> succeedDirs = Lists.newArrayList();
     try {
       for (File dataDir : dataDirs) {
@@ -220,7 +220,7 @@ public class BlockPoolSliceStorage extends Storage {
                   "attempt to load an used block storage: " + dataDir);
         }
         final StorageDirectory sd = loadStorageDirectory(
-            nsInfo, dataDir, startOpt, callables, conf);
+            nsInfo, dataDir, location, startOpt, callables, conf);
         succeedDirs.add(sd);
       }
     } catch (IOException e) {
@@ -244,12 +244,12 @@ public class BlockPoolSliceStorage extends Storage {
    * @throws IOException on error
    */
   List<StorageDirectory> recoverTransitionRead(NamespaceInfo nsInfo,
-      Collection<File> dataDirs, StartupOption startOpt,
-      List<Callable<StorageDirectory>> callables, Configuration conf)
-          throws IOException {
+      Collection<File> dataDirs, StorageLocation location,
+      StartupOption startOpt, List<Callable<StorageDirectory>> callables,
+      Configuration conf) throws IOException {
     LOG.info("Analyzing storage directories for bpid " + nsInfo.getBlockPoolID());
     final List<StorageDirectory> loaded = loadBpStorageDirectories(
-        nsInfo, dataDirs, startOpt, callables, conf);
+        nsInfo, dataDirs, location, startOpt, callables, conf);
     for (StorageDirectory sd : loaded) {
       addStorageDir(sd);
     }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java

@@ -201,17 +201,17 @@ public class BlockScanner {
       FsVolumeSpi volume = ref.getVolume();
       if (!isEnabled()) {
         LOG.debug("Not adding volume scanner for {}, because the block " +
-            "scanner is disabled.", volume.getBasePath());
+            "scanner is disabled.", volume);
         return;
       }
       VolumeScanner scanner = scanners.get(volume.getStorageID());
       if (scanner != null) {
         LOG.error("Already have a scanner for volume {}.",
-            volume.getBasePath());
+            volume);
         return;
       }
       LOG.debug("Adding scanner for volume {} (StorageID {})",
-          volume.getBasePath(), volume.getStorageID());
+          volume, volume.getStorageID());
       scanner = new VolumeScanner(conf, datanode, ref);
       scanner.start();
       scanners.put(volume.getStorageID(), scanner);
@@ -245,7 +245,7 @@ public class BlockScanner {
       return;
     }
     LOG.info("Removing scanner for volume {} (StorageID {})",
-        volume.getBasePath(), volume.getStorageID());
+        volume, volume.getStorageID());
     scanner.shutdown();
     scanners.remove(volume.getStorageID());
     Uninterruptibles.joinUninterruptibly(scanner, 5, TimeUnit.MINUTES);

+ 12 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -60,7 +60,6 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
-import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -81,7 +80,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -802,11 +800,7 @@ public class DataNode extends ReconfigurableBase
     if (locations.isEmpty()) {
       return;
     }
-    Set<File> volumesToRemove = new HashSet<>();
-    for (StorageLocation loc : locations) {
-      volumesToRemove.add(loc.getFile().getAbsoluteFile());
-    }
-    removeVolumes(volumesToRemove, true);
+    removeVolumes(locations, true);
   }
 
   /**
@@ -825,26 +819,22 @@ public class DataNode extends ReconfigurableBase
    * @throws IOException
    */
   private synchronized void removeVolumes(
-      final Set<File> absoluteVolumePaths, boolean clearFailure)
+      final Collection<StorageLocation> storageLocations, boolean clearFailure)
       throws IOException {
-    for (File vol : absoluteVolumePaths) {
-      Preconditions.checkArgument(vol.isAbsolute());
-    }
-
-    if (absoluteVolumePaths.isEmpty()) {
+    if (storageLocations.isEmpty()) {
       return;
     }
 
     LOG.info(String.format("Deactivating volumes (clear failure=%b): %s",
-        clearFailure, Joiner.on(",").join(absoluteVolumePaths)));
+        clearFailure, Joiner.on(",").join(storageLocations)));
 
     IOException ioe = null;
     // Remove volumes and block infos from FsDataset.
-    data.removeVolumes(absoluteVolumePaths, clearFailure);
+    data.removeVolumes(storageLocations, clearFailure);
 
     // Remove volumes from DataStorage.
     try {
-      storage.removeVolumes(absoluteVolumePaths);
+      storage.removeVolumes(storageLocations);
     } catch (IOException e) {
       ioe = e;
     }
@@ -852,7 +842,7 @@ public class DataNode extends ReconfigurableBase
     // Set configuration and dataDirs to reflect volume changes.
     for (Iterator<StorageLocation> it = dataDirs.iterator(); it.hasNext(); ) {
       StorageLocation loc = it.next();
-      if (absoluteVolumePaths.contains(loc.getFile().getAbsoluteFile())) {
+      if (storageLocations.contains(loc)) {
         it.remove();
       }
     }
@@ -3292,18 +3282,18 @@ public class DataNode extends ReconfigurableBase
    * Check the disk error
    */
   private void checkDiskError() {
-    Set<File> unhealthyDataDirs = data.checkDataDir();
-    if (unhealthyDataDirs != null && !unhealthyDataDirs.isEmpty()) {
+    Set<StorageLocation> unhealthyLocations = data.checkDataDir();
+    if (unhealthyLocations != null && !unhealthyLocations.isEmpty()) {
       try {
         // Remove all unhealthy volumes from DataNode.
-        removeVolumes(unhealthyDataDirs, false);
+        removeVolumes(unhealthyLocations, false);
       } catch (IOException e) {
         LOG.warn("Error occurred when removing unhealthy storage dirs: "
             + e.getMessage(), e);
       }
       StringBuilder sb = new StringBuilder("DataNode failed volumes:");
-      for (File dataDir : unhealthyDataDirs) {
-        sb.append(dataDir.getAbsolutePath() + ";");
+      for (StorageLocation location : unhealthyLocations) {
+        sb.append(location + ";");
       }
       handleDiskError(sb.toString());
     }

+ 21 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -263,9 +263,10 @@ public class DataStorage extends Storage {
   }
 
   private StorageDirectory loadStorageDirectory(DataNode datanode,
-      NamespaceInfo nsInfo, File dataDir, StartupOption startOpt,
-      List<Callable<StorageDirectory>> callables) throws IOException {
-    StorageDirectory sd = new StorageDirectory(dataDir, null, false);
+      NamespaceInfo nsInfo, File dataDir, StorageLocation location,
+      StartupOption startOpt, List<Callable<StorageDirectory>> callables)
+          throws IOException {
+    StorageDirectory sd = new StorageDirectory(dataDir, null, false, location);
     try {
       StorageState curState = sd.analyzeStorage(startOpt, this, true);
       // sd is locked but not opened
@@ -310,7 +311,7 @@ public class DataStorage extends Storage {
    * builder later.
    *
    * @param datanode DataNode object.
-   * @param volume the root path of a storage directory.
+   * @param location the StorageLocation for the storage directory.
    * @param nsInfos an array of namespace infos.
    * @return a VolumeBuilder that holds the metadata of this storage directory
    * and can be added to DataStorage later.
@@ -318,8 +319,10 @@ public class DataStorage extends Storage {
    *
    * Note that if there is IOException, the state of DataStorage is not modified.
    */
-  public VolumeBuilder prepareVolume(DataNode datanode, File volume,
-      List<NamespaceInfo> nsInfos) throws IOException {
+  public VolumeBuilder prepareVolume(DataNode datanode,
+      StorageLocation location, List<NamespaceInfo> nsInfos)
+          throws IOException {
+    File volume = location.getFile();
     if (containsStorageDir(volume)) {
       final String errorMessage = "Storage directory is in use";
       LOG.warn(errorMessage + ".");
@@ -327,7 +330,8 @@ public class DataStorage extends Storage {
     }
 
     StorageDirectory sd = loadStorageDirectory(
-        datanode, nsInfos.get(0), volume, StartupOption.HOTSWAP, null);
+        datanode, nsInfos.get(0), volume, location,
+        StartupOption.HOTSWAP, null);
     VolumeBuilder builder =
         new VolumeBuilder(this, sd);
     for (NamespaceInfo nsInfo : nsInfos) {
@@ -338,7 +342,8 @@ public class DataStorage extends Storage {
 
       final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
       final List<StorageDirectory> dirs = bpStorage.loadBpStorageDirectories(
-          nsInfo, bpDataDirs, StartupOption.HOTSWAP, null, datanode.getConf());
+          nsInfo, bpDataDirs, location, StartupOption.HOTSWAP,
+          null, datanode.getConf());
       builder.addBpStorageDirectories(nsInfo.getBlockPoolID(), dirs);
     }
     return builder;
@@ -407,7 +412,7 @@ public class DataStorage extends Storage {
           final List<Callable<StorageDirectory>> callables
               = Lists.newArrayList();
           final StorageDirectory sd = loadStorageDirectory(
-              datanode, nsInfo, root, startOpt, callables);
+              datanode, nsInfo, root, dataDir, startOpt, callables);
           if (callables.isEmpty()) {
             addStorageDir(sd);
             success.add(dataDir);
@@ -458,7 +463,8 @@ public class DataStorage extends Storage {
 
         final List<Callable<StorageDirectory>> callables = Lists.newArrayList();
         final List<StorageDirectory> dirs = bpStorage.recoverTransitionRead(
-            nsInfo, bpDataDirs, startOpt, callables, datanode.getConf());
+            nsInfo, bpDataDirs, dataDir, startOpt,
+            callables, datanode.getConf());
         if (callables.isEmpty()) {
           for(StorageDirectory sd : dirs) {
             success.add(sd);
@@ -498,9 +504,10 @@ public class DataStorage extends Storage {
    * @param dirsToRemove a set of storage directories to be removed.
    * @throws IOException if I/O error when unlocking storage directory.
    */
-  synchronized void removeVolumes(final Set<File> dirsToRemove)
+  synchronized void removeVolumes(
+      final Collection<StorageLocation> storageLocations)
       throws IOException {
-    if (dirsToRemove.isEmpty()) {
+    if (storageLocations.isEmpty()) {
       return;
     }
 
@@ -508,7 +515,8 @@ public class DataStorage extends Storage {
     for (Iterator<StorageDirectory> it = this.storageDirs.iterator();
          it.hasNext(); ) {
       StorageDirectory sd = it.next();
-      if (dirsToRemove.contains(sd.getRoot())) {
+      StorageLocation sdLocation = sd.getStorageLocation();
+      if (storageLocations.contains(sdLocation)) {
         // Remove the block pool level storage first.
         for (Map.Entry<String, BlockPoolSliceStorage> entry :
             this.bpStorageMap.entrySet()) {

+ 6 - 314
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -22,7 +22,6 @@ import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -37,9 +36,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -47,10 +43,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.Time;
@@ -209,200 +204,6 @@ public class DirectoryScanner implements Runnable {
     }
   }
 
-  /**
-   * Tracks the files and other information related to a block on the disk
-   * Missing file is indicated by setting the corresponding member
-   * to null.
-   * 
-   * Because millions of these structures may be created, we try to save
-   * memory here.  So instead of storing full paths, we store path suffixes.
-   * The block file, if it exists, will have a path like this:
-   * <volume_base_path>/<block_path>
-   * So we don't need to store the volume path, since we already know what the
-   * volume is.
-   * 
-   * The metadata file, if it exists, will have a path like this:
-   * <volume_base_path>/<block_path>_<genstamp>.meta
-   * So if we have a block file, there isn't any need to store the block path
-   * again.
-   * 
-   * The accessor functions take care of these manipulations.
-   */
-  static class ScanInfo implements Comparable<ScanInfo> {
-    private final long blockId;
-    
-    /**
-     * The block file path, relative to the volume's base directory.
-     * If there was no block file found, this may be null. If 'vol'
-     * is null, then this is the full path of the block file.
-     */
-    private final String blockSuffix;
-    
-    /**
-     * The suffix of the meta file path relative to the block file.
-     * If blockSuffix is null, then this will be the entire path relative
-     * to the volume base directory, or an absolute path if vol is also
-     * null.
-     */
-    private final String metaSuffix;
-
-    private final FsVolumeSpi volume;
-
-    /**
-     * Get the file's length in async block scan
-     */
-    private final long blockFileLength;
-
-    private final static Pattern CONDENSED_PATH_REGEX =
-        Pattern.compile("(?<!^)(\\\\|/){2,}");
-    
-    private final static String QUOTED_FILE_SEPARATOR = 
-        Matcher.quoteReplacement(File.separator);
-    
-    /**
-     * Get the most condensed version of the path.
-     *
-     * For example, the condensed version of /foo//bar is /foo/bar
-     * Unlike {@link File#getCanonicalPath()}, this will never perform I/O
-     * on the filesystem.
-     *
-     * @param path the path to condense
-     * @return the condensed path
-     */
-    private static String getCondensedPath(String path) {
-      return CONDENSED_PATH_REGEX.matcher(path).
-          replaceAll(QUOTED_FILE_SEPARATOR);
-    }
-
-    /**
-     * Get a path suffix.
-     *
-     * @param f            The file to get the suffix for.
-     * @param prefix       The prefix we're stripping off.
-     *
-     * @return             A suffix such that prefix + suffix = path to f
-     */
-    private static String getSuffix(File f, String prefix) {
-      String fullPath = getCondensedPath(f.getAbsolutePath());
-      if (fullPath.startsWith(prefix)) {
-        return fullPath.substring(prefix.length());
-      }
-      throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
-    }
-
-    /**
-     * Create a ScanInfo object for a block. This constructor will examine
-     * the block data and meta-data files.
-     *
-     * @param blockId the block ID
-     * @param blockFile the path to the block data file
-     * @param metaFile the path to the block meta-data file
-     * @param vol the volume that contains the block
-     */
-    ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) {
-      this.blockId = blockId;
-      String condensedVolPath = vol == null ? null :
-        getCondensedPath(vol.getBasePath());
-      this.blockSuffix = blockFile == null ? null :
-        getSuffix(blockFile, condensedVolPath);
-      this.blockFileLength = (blockFile != null) ? blockFile.length() : 0; 
-      if (metaFile == null) {
-        this.metaSuffix = null;
-      } else if (blockFile == null) {
-        this.metaSuffix = getSuffix(metaFile, condensedVolPath);
-      } else {
-        this.metaSuffix = getSuffix(metaFile,
-            condensedVolPath + blockSuffix);
-      }
-      this.volume = vol;
-    }
-
-    /**
-     * Returns the block data file.
-     *
-     * @return the block data file
-     */
-    File getBlockFile() {
-      return (blockSuffix == null) ? null :
-        new File(volume.getBasePath(), blockSuffix);
-    }
-
-    /**
-     * Return the length of the data block. The length returned is the length
-     * cached when this object was created.
-     *
-     * @return the length of the data block
-     */
-    long getBlockFileLength() {
-      return blockFileLength;
-    }
-
-    /**
-     * Returns the block meta data file or null if there isn't one.
-     *
-     * @return the block meta data file
-     */
-    File getMetaFile() {
-      if (metaSuffix == null) {
-        return null;
-      } else if (blockSuffix == null) {
-        return new File(volume.getBasePath(), metaSuffix);
-      } else {
-        return new File(volume.getBasePath(), blockSuffix + metaSuffix);
-      }
-    }
-
-    /**
-     * Returns the block ID.
-     *
-     * @return the block ID
-     */
-    long getBlockId() {
-      return blockId;
-    }
-
-    /**
-     * Returns the volume that contains the block that this object describes.
-     *
-     * @return the volume
-     */
-    FsVolumeSpi getVolume() {
-      return volume;
-    }
-
-    @Override // Comparable
-    public int compareTo(ScanInfo b) {
-      if (blockId < b.blockId) {
-        return -1;
-      } else if (blockId == b.blockId) {
-        return 0;
-      } else {
-        return 1;
-      }
-    }
-
-    @Override // Object
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (!(o instanceof ScanInfo)) {
-        return false;
-      }
-      return blockId == ((ScanInfo) o).blockId;
-    }
-
-    @Override // Object
-    public int hashCode() {
-      return (int)(blockId^(blockId>>>32));
-    }
-
-    public long getGenStamp() {
-      return metaSuffix != null ? Block.getGenerationStamp(
-          getMetaFile().getName()) : 
-            HdfsConstants.GRANDFATHER_GENERATION_STAMP;
-    }
-  }
 
   /**
    * Create a new directory scanner, but don't cycle it running yet.
@@ -644,7 +445,7 @@ public class DirectoryScanner implements Runnable {
             // There may be multiple on-disk records for the same block, don't increment
             // the memory record pointer if so.
             ScanInfo nextInfo = blockpoolReport[Math.min(d, blockpoolReport.length - 1)];
-            if (nextInfo.getBlockId() != info.blockId) {
+            if (nextInfo.getBlockId() != info.getBlockId()) {
               ++m;
             }
           } else {
@@ -762,19 +563,6 @@ public class DirectoryScanner implements Runnable {
     return list.toSortedArrays();
   }
 
-  /**
-   * Helper method to determine if a file name is consistent with a block.
-   * meta-data file
-   *
-   * @param blockId the block ID
-   * @param metaFile the file to check
-   * @return whether the file name is a block meta-data file name
-   */
-  private static boolean isBlockMetaFile(String blockId, String metaFile) {
-    return metaFile.startsWith(blockId)
-        && metaFile.endsWith(Block.METADATA_EXTENSION);
-  }
-
   /**
    * The ReportCompiler class encapsulates the process of searching a datanode's
    * disks for block information.  It operates by performing a DFS of the
@@ -784,7 +572,7 @@ public class DirectoryScanner implements Runnable {
    * ScanInfo object for it and adds that object to its report list.  The report
    * list is returned by the {@link #call()} method.
    */
-  private class ReportCompiler implements Callable<ScanInfoPerBlockPool> {
+  public class ReportCompiler implements Callable<ScanInfoPerBlockPool> {
     private final FsVolumeSpi volume;
     private final DataNode datanode;
     // Variable for tracking time spent running for throttling purposes
@@ -816,14 +604,12 @@ public class DirectoryScanner implements Runnable {
       ScanInfoPerBlockPool result = new ScanInfoPerBlockPool(bpList.length);
       for (String bpid : bpList) {
         LinkedList<ScanInfo> report = new LinkedList<>();
-        File bpFinalizedDir = volume.getFinalizedDir(bpid);
 
         perfTimer.start();
         throttleTimer.start();
 
         try {
-          result.put(bpid,
-              compileReport(volume, bpFinalizedDir, bpFinalizedDir, report));
+          result.put(bpid, volume.compileReport(bpid, report, this));
         } catch (InterruptedException ex) {
           // Exit quickly and flag the scanner to do the same
           result = null;
@@ -833,107 +619,13 @@ public class DirectoryScanner implements Runnable {
       return result;
     }
 
-    /**
-     * Compile a list of {@link ScanInfo} for the blocks in the directory
-     * given by {@code dir}.
-     *
-     * @param vol the volume that contains the directory to scan
-     * @param bpFinalizedDir the root directory of the directory to scan
-     * @param dir the directory to scan
-     * @param report the list onto which blocks reports are placed
-     */
-    private LinkedList<ScanInfo> compileReport(FsVolumeSpi vol,
-        File bpFinalizedDir, File dir, LinkedList<ScanInfo> report)
-        throws InterruptedException {
-
-      throttle();
-
-      List <String> fileNames;
-      try {
-        fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
-      } catch (IOException ioe) {
-        LOG.warn("Exception occured while compiling report: ", ioe);
-        // Initiate a check on disk failure.
-        datanode.checkDiskErrorAsync();
-        // Ignore this directory and proceed.
-        return report;
-      }
-      Collections.sort(fileNames);
-
-      /*
-       * Assumption: In the sorted list of files block file appears immediately
-       * before block metadata file. This is true for the current naming
-       * convention for block file blk_<blockid> and meta file
-       * blk_<blockid>_<genstamp>.meta
-       */
-      for (int i = 0; i < fileNames.size(); i++) {
-        // Make sure this thread can make a timely exit. With a low throttle
-        // rate, completing a run can take a looooong time.
-        if (Thread.interrupted()) {
-          throw new InterruptedException();
-        }
-
-        File file = new File(dir, fileNames.get(i));
-        if (file.isDirectory()) {
-          compileReport(vol, bpFinalizedDir, file, report);
-          continue;
-        }
-        if (!Block.isBlockFilename(file)) {
-          if (isBlockMetaFile(Block.BLOCK_FILE_PREFIX, file.getName())) {
-            long blockId = Block.getBlockId(file.getName());
-            verifyFileLocation(file.getParentFile(), bpFinalizedDir,
-                blockId);
-            report.add(new ScanInfo(blockId, null, file, vol));
-          }
-          continue;
-        }
-        File blockFile = file;
-        long blockId = Block.filename2id(file.getName());
-        File metaFile = null;
-
-        // Skip all the files that start with block name until
-        // getting to the metafile for the block
-        while (i + 1 < fileNames.size()) {
-          File blkMetaFile = new File(dir, fileNames.get(i + 1));
-          if (!(blkMetaFile.isFile()
-              && blkMetaFile.getName().startsWith(blockFile.getName()))) {
-            break;
-          }
-          i++;
-          if (isBlockMetaFile(blockFile.getName(), blkMetaFile.getName())) {
-            metaFile = blkMetaFile;
-            break;
-          }
-        }
-        verifyFileLocation(blockFile, bpFinalizedDir, blockId);
-        report.add(new ScanInfo(blockId, blockFile, metaFile, vol));
-      }
-      return report;
-    }
-
-    /**
-     * Verify whether the actual directory location of block file has the
-     * expected directory path computed using its block ID.
-     */
-    private void verifyFileLocation(File actualBlockFile,
-        File bpFinalizedDir, long blockId) {
-      File expectedBlockDir =
-          DatanodeUtil.idToBlockDir(bpFinalizedDir, blockId);
-      File actualBlockDir = actualBlockFile.getParentFile();
-      if (actualBlockDir.compareTo(expectedBlockDir) != 0) {
-        LOG.warn("Block: " + blockId +
-            " found in invalid directory.  Expected directory: " +
-            expectedBlockDir + ".  Actual directory: " + actualBlockDir);
-      }
-    }
-
     /**
      * Called by the thread before each potential disk scan so that a pause
      * can be optionally inserted to limit the number of scans per second.
      * The limit is controlled by
      * {@link DFSConfigKeys#DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY}.
      */
-    private void throttle() throws InterruptedException {
+    public void throttle() throws InterruptedException {
       accumulateTimeRunning();
 
       if ((throttleLimitMsPerSec < 1000) &&
@@ -963,7 +655,7 @@ public class DirectoryScanner implements Runnable {
     }
   }
 
-  private enum BlockDirFilter implements FilenameFilter {
+  public enum BlockDirFilter implements FilenameFilter {
     INSTANCE;
 
     @Override

+ 12 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java

@@ -500,7 +500,8 @@ public class DiskBalancer {
         references = this.dataset.getFsVolumeReferences();
         for (int ndx = 0; ndx < references.size(); ndx++) {
           FsVolumeSpi vol = references.get(ndx);
-          storageIDToVolBasePathMap.put(vol.getStorageID(), vol.getBasePath());
+          storageIDToVolBasePathMap.put(vol.getStorageID(),
+              vol.getBaseURI().getPath());
         }
         references.close();
       }
@@ -1023,7 +1024,7 @@ public class DiskBalancer {
         openPoolIters(source, poolIters);
         if (poolIters.size() == 0) {
           LOG.error("No block pools found on volume. volume : {}. Exiting.",
-              source.getBasePath());
+              source.getBaseURI());
           return;
         }
 
@@ -1033,17 +1034,16 @@ public class DiskBalancer {
             // Check for the max error count constraint.
             if (item.getErrorCount() > getMaxError(item)) {
               LOG.error("Exceeded the max error count. source {}, dest: {} " +
-                      "error count: {}", source.getBasePath(),
-                  dest.getBasePath(), item.getErrorCount());
-              this.setExitFlag();
-              continue;
+                      "error count: {}", source.getBaseURI(),
+                  dest.getBaseURI(), item.getErrorCount());
+              break;
             }
 
             // Check for the block tolerance constraint.
             if (isCloseEnough(item)) {
               LOG.info("Copy from {} to {} done. copied {} bytes and {} " +
                       "blocks.",
-                  source.getBasePath(), dest.getBasePath(),
+                  source.getBaseURI(), dest.getBaseURI(),
                   item.getBytesCopied(), item.getBlocksCopied());
               this.setExitFlag();
               continue;
@@ -1053,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.getBaseURI(), dest.getBaseURI());
               this.setExitFlag();
               continue;
             }
@@ -1081,14 +1081,13 @@ public class DiskBalancer {
               // exiting here.
               LOG.error("Destination volume: {} does not have enough space to" +
                   " accommodate a block. Block Size: {} Exiting from" +
-                  " copyBlocks.", dest.getBasePath(), block.getNumBytes());
-              this.setExitFlag();
-              continue;
+                  " copyBlocks.", dest.getBaseURI(), block.getNumBytes());
+              break;
             }
 
             LOG.debug("Moved block with size {} from  {} to {}",
-                block.getNumBytes(), source.getBasePath(),
-                dest.getBasePath());
+                block.getNumBytes(), source.getBaseURI(),
+                dest.getBaseURI());
 
             // Check for the max throughput constraint.
             // We sleep here to keep the promise that we will not

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

@@ -39,8 +39,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.io.IOUtils;

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

@@ -25,8 +25,8 @@ import java.net.URI;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.util.LightWeightResizableGSet;

+ 27 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.util.StringUtils;
 
+
 /**
  * Encapsulates the URI and storage medium that together describe a
  * storage directory.
@@ -37,7 +38,7 @@ import org.apache.hadoop.util.StringUtils;
  *
  */
 @InterfaceAudience.Private
-public class StorageLocation {
+public class StorageLocation implements Comparable<StorageLocation>{
   final StorageType storageType;
   final File file;
 
@@ -104,16 +105,37 @@ public class StorageLocation {
 
   @Override
   public boolean equals(Object obj) {
-    if (obj == this) {
-      return true;
-    } else if (obj == null || !(obj instanceof StorageLocation)) {
+    if (obj == null || !(obj instanceof StorageLocation)) {
       return false;
     }
-    return toString().equals(obj.toString());
+    int comp = compareTo((StorageLocation) obj);
+    return comp == 0;
   }
 
   @Override
   public int hashCode() {
     return toString().hashCode();
   }
+
+  @Override
+  public int compareTo(StorageLocation obj) {
+    if (obj == this) {
+      return 0;
+    } else if (obj == null) {
+      return -1;
+    }
+
+    StorageLocation otherStorage = (StorageLocation) obj;
+    if (this.getFile() != null && otherStorage.getFile() != null) {
+      return this.getFile().getAbsolutePath().compareTo(
+          otherStorage.getFile().getAbsolutePath());
+    } else if (this.getFile() == null && otherStorage.getFile() == null) {
+      return this.storageType.compareTo(otherStorage.getStorageType());
+    } else if (this.getFile() == null) {
+      return -1;
+    } else {
+      return 1;
+    }
+
+  }
 }

+ 13 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java

@@ -217,7 +217,7 @@ public class VolumeScanner extends Thread {
 
   public void printStats(StringBuilder p) {
     p.append(String.format("Block scanner information for volume %s with base" +
-        " path %s%n", volume.getStorageID(), volume.getBasePath()));
+        " path %s%n", volume.getStorageID(), volume));
     synchronized (stats) {
       p.append(String.format("Bytes verified in last hour       : %57d%n",
           stats.bytesScannedInPastHour));
@@ -253,20 +253,20 @@ public class VolumeScanner extends Thread {
 
     public void setup(VolumeScanner scanner) {
       LOG.trace("Starting VolumeScanner {}",
-          scanner.volume.getBasePath());
+          scanner.volume);
       this.scanner = scanner;
     }
 
     public void handle(ExtendedBlock block, IOException e) {
       FsVolumeSpi volume = scanner.volume;
       if (e == null) {
-        LOG.trace("Successfully scanned {} on {}", block, volume.getBasePath());
+        LOG.trace("Successfully scanned {} on {}", block, volume);
         return;
       }
       // If the block does not exist anymore, then it's not an error.
       if (!volume.getDataset().contains(block)) {
         LOG.debug("Volume {}: block {} is no longer in the dataset.",
-            volume.getBasePath(), block);
+            volume, block);
         return;
       }
       // If the block exists, the exception may due to a race with write:
@@ -278,11 +278,10 @@ public class VolumeScanner extends Thread {
       if (e instanceof FileNotFoundException ) {
         LOG.info("Volume {}: verification failed for {} because of " +
                 "FileNotFoundException.  This may be due to a race with write.",
-            volume.getBasePath(), block);
+            volume, block);
         return;
       }
-      LOG.warn("Reporting bad " + block + " with volume "
-          + volume.getBasePath(), e);
+      LOG.warn("Reporting bad {} on {}", block, volume);
       try {
         scanner.datanode.reportBadBlocks(block, volume);
       } catch (IOException ie) {
@@ -305,7 +304,7 @@ public class VolumeScanner extends Thread {
       handler = new ScanResultHandler();
     }
     this.resultHandler = handler;
-    setName("VolumeScannerThread(" + volume.getBasePath() + ")");
+    setName("VolumeScannerThread(" + volume + ")");
     setDaemon(true);
   }
 
@@ -376,7 +375,7 @@ public class VolumeScanner extends Thread {
       BlockIterator iter = blockIters.get(idx);
       if (!iter.atEnd()) {
         LOG.info("Now scanning bpid {} on volume {}",
-            iter.getBlockPoolId(), volume.getBasePath());
+            iter.getBlockPoolId(), volume);
         curBlockIter = iter;
         return 0L;
       }
@@ -385,7 +384,7 @@ public class VolumeScanner extends Thread {
       if (waitMs <= 0) {
         iter.rewind();
         LOG.info("Now rescanning bpid {} on volume {}, after more than " +
-            "{} hour(s)", iter.getBlockPoolId(), volume.getBasePath(),
+            "{} hour(s)", iter.getBlockPoolId(), volume,
             TimeUnit.HOURS.convert(conf.scanPeriodMs, TimeUnit.MILLISECONDS));
         curBlockIter = iter;
         return 0L;
@@ -416,16 +415,16 @@ public class VolumeScanner extends Thread {
           cblock.getBlockPoolId(), cblock.getBlockId());
       if (b == null) {
         LOG.info("Replica {} was not found in the VolumeMap for volume {}",
-            cblock, volume.getBasePath());
+            cblock, volume);
       } else {
         block = new ExtendedBlock(cblock.getBlockPoolId(), b);
       }
     } catch (FileNotFoundException e) {
       LOG.info("FileNotFoundException while finding block {} on volume {}",
-          cblock, volume.getBasePath());
+          cblock, volume);
     } catch (IOException e) {
       LOG.warn("I/O error while finding block {} on volume {}",
-            cblock, volume.getBasePath());
+            cblock, volume);
     }
     if (block == null) {
       return -1; // block not found.
@@ -642,7 +641,7 @@ public class VolumeScanner extends Thread {
 
   @Override
   public String toString() {
-    return "VolumeScanner(" + volume.getBasePath() +
+    return "VolumeScanner(" + volume +
         ", " + volume.getStorageID() + ")";
   }
 

+ 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;
     }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java

@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -206,7 +207,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param clearFailure set true to clear the failure information about the
    *                     volumes.
    */
-  void removeVolumes(Set<File> volumes, boolean clearFailure);
+  void removeVolumes(Collection<StorageLocation> volumes, boolean clearFailure);
 
   /** @return a storage with the given storage ID */
   DatanodeStorage getStorage(final String storageUuid);
@@ -482,7 +483,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
      * Check if all the data directories are healthy
      * @return A set of unhealthy data directories.
      */
-  Set<File> checkDataDir();
+  Set<StorageLocation> checkDataDir();
 
   /**
    * Shutdown the FSDataset

+ 228 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java

@@ -20,10 +20,20 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.nio.channels.ClosedChannelException;
+import java.util.LinkedList;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 
 /**
  * This is an interface for the underlying volume.
@@ -48,14 +58,14 @@ public interface FsVolumeSpi {
   long getAvailable() throws IOException;
 
   /** @return the base path to the volume */
-  String getBasePath();
+  URI getBaseURI();
 
-  /** @return the path to the volume */
-  String getPath(String bpid) throws IOException;
+  DF getUsageStats(Configuration conf);
 
-  /** @return the directory for the finalized blocks in the block pool. */
-  File getFinalizedDir(String bpid) throws IOException;
-  
+  /** @return the {@link StorageLocation} to the volume */
+  StorageLocation getStorageLocation();
+
+  /** @return the {@link StorageType} of the volume */
   StorageType getStorageType();
 
   /** Returns true if the volume is NOT backed by persistent storage. */
@@ -186,4 +196,216 @@ public interface FsVolumeSpi {
    * Get the FSDatasetSpi which this volume is a part of.
    */
   FsDatasetSpi getDataset();
+
+  /**
+   * Tracks the files and other information related to a block on the disk
+   * Missing file is indicated by setting the corresponding member
+   * to null.
+   *
+   * Because millions of these structures may be created, we try to save
+   * memory here.  So instead of storing full paths, we store path suffixes.
+   * The block file, if it exists, will have a path like this:
+   * <volume_base_path>/<block_path>
+   * So we don't need to store the volume path, since we already know what the
+   * volume is.
+   *
+   * The metadata file, if it exists, will have a path like this:
+   * <volume_base_path>/<block_path>_<genstamp>.meta
+   * So if we have a block file, there isn't any need to store the block path
+   * again.
+   *
+   * The accessor functions take care of these manipulations.
+   */
+  public static class ScanInfo implements Comparable<ScanInfo> {
+    private final long blockId;
+
+    /**
+     * The block file path, relative to the volume's base directory.
+     * If there was no block file found, this may be null. If 'vol'
+     * is null, then this is the full path of the block file.
+     */
+    private final String blockSuffix;
+
+    /**
+     * The suffix of the meta file path relative to the block file.
+     * If blockSuffix is null, then this will be the entire path relative
+     * to the volume base directory, or an absolute path if vol is also
+     * null.
+     */
+    private final String metaSuffix;
+
+    private final FsVolumeSpi volume;
+
+    /**
+     * Get the file's length in async block scan
+     */
+    private final long blockFileLength;
+
+    private final static Pattern CONDENSED_PATH_REGEX =
+        Pattern.compile("(?<!^)(\\\\|/){2,}");
+
+    private final static String QUOTED_FILE_SEPARATOR =
+        Matcher.quoteReplacement(File.separator);
+
+    /**
+     * Get the most condensed version of the path.
+     *
+     * For example, the condensed version of /foo//bar is /foo/bar
+     * Unlike {@link File#getCanonicalPath()}, this will never perform I/O
+     * on the filesystem.
+     *
+     * @param path the path to condense
+     * @return the condensed path
+     */
+    private static String getCondensedPath(String path) {
+      return CONDENSED_PATH_REGEX.matcher(path).
+          replaceAll(QUOTED_FILE_SEPARATOR);
+    }
+
+    /**
+     * Get a path suffix.
+     *
+     * @param f            The file to get the suffix for.
+     * @param prefix       The prefix we're stripping off.
+     *
+     * @return             A suffix such that prefix + suffix = path to f
+     */
+    private static String getSuffix(File f, String prefix) {
+      String fullPath = getCondensedPath(f.getAbsolutePath());
+      if (fullPath.startsWith(prefix)) {
+        return fullPath.substring(prefix.length());
+      }
+      throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
+    }
+
+    /**
+     * Create a ScanInfo object for a block. This constructor will examine
+     * the block data and meta-data files.
+     *
+     * @param blockId the block ID
+     * @param blockFile the path to the block data file
+     * @param metaFile the path to the block meta-data file
+     * @param vol the volume that contains the block
+     */
+    public ScanInfo(long blockId, File blockFile, File metaFile,
+        FsVolumeSpi vol) {
+      this.blockId = blockId;
+      String condensedVolPath =
+          (vol == null || vol.getBaseURI() == null) ? null :
+            getCondensedPath(new File(vol.getBaseURI()).getAbsolutePath());
+      this.blockSuffix = blockFile == null ? null :
+        getSuffix(blockFile, condensedVolPath);
+      this.blockFileLength = (blockFile != null) ? blockFile.length() : 0;
+      if (metaFile == null) {
+        this.metaSuffix = null;
+      } else if (blockFile == null) {
+        this.metaSuffix = getSuffix(metaFile, condensedVolPath);
+      } else {
+        this.metaSuffix = getSuffix(metaFile,
+            condensedVolPath + blockSuffix);
+      }
+      this.volume = vol;
+    }
+
+    /**
+     * Returns the block data file.
+     *
+     * @return the block data file
+     */
+    public File getBlockFile() {
+      return (blockSuffix == null) ? null :
+        new File(new File(volume.getBaseURI()).getAbsolutePath(), blockSuffix);
+    }
+
+    /**
+     * Return the length of the data block. The length returned is the length
+     * cached when this object was created.
+     *
+     * @return the length of the data block
+     */
+    public long getBlockFileLength() {
+      return blockFileLength;
+    }
+
+    /**
+     * Returns the block meta data file or null if there isn't one.
+     *
+     * @return the block meta data file
+     */
+    public File getMetaFile() {
+      if (metaSuffix == null) {
+        return null;
+      } else if (blockSuffix == null) {
+        return new File(new File(volume.getBaseURI()).getAbsolutePath(),
+            metaSuffix);
+      } else {
+        return new File(new File(volume.getBaseURI()).getAbsolutePath(),
+            blockSuffix + metaSuffix);
+      }
+    }
+
+    /**
+     * Returns the block ID.
+     *
+     * @return the block ID
+     */
+    public long getBlockId() {
+      return blockId;
+    }
+
+    /**
+     * Returns the volume that contains the block that this object describes.
+     *
+     * @return the volume
+     */
+    public FsVolumeSpi getVolume() {
+      return volume;
+    }
+
+    @Override // Comparable
+    public int compareTo(ScanInfo b) {
+      if (blockId < b.blockId) {
+        return -1;
+      } else if (blockId == b.blockId) {
+        return 0;
+      } else {
+        return 1;
+      }
+    }
+
+    @Override // Object
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof ScanInfo)) {
+        return false;
+      }
+      return blockId == ((ScanInfo) o).blockId;
+    }
+
+    @Override // Object
+    public int hashCode() {
+      return (int)(blockId^(blockId>>>32));
+    }
+
+    public long getGenStamp() {
+      return metaSuffix != null ? Block.getGenerationStamp(
+          getMetaFile().getName()) :
+            HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    }
+  }
+
+  /**
+   * Compile a list of {@link ScanInfo} for the blocks in
+   * the block pool with id {@code bpid}.
+   *
+   * @param bpid block pool id to scan
+   * @param report the list onto which blocks reports are placed
+   * @param reportCompiler
+   * @throws IOException
+   */
+  LinkedList<ScanInfo> compileReport(String bpid,
+      LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
+      throws InterruptedException, IOException;
 }

+ 23 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java

@@ -71,8 +71,8 @@ class FsDatasetAsyncDiskService {
   private final DataNode datanode;
   private final FsDatasetImpl fsdatasetImpl;
   private final ThreadGroup threadGroup;
-  private Map<File, ThreadPoolExecutor> executors
-      = new HashMap<File, ThreadPoolExecutor>();
+  private Map<String, ThreadPoolExecutor> executors
+      = new HashMap<String, ThreadPoolExecutor>();
   private Map<String, Set<Long>> deletedBlockIds 
       = new HashMap<String, Set<Long>>();
   private static final int MAX_DELETED_BLOCKS = 64;
@@ -91,7 +91,7 @@ class FsDatasetAsyncDiskService {
     this.threadGroup = new ThreadGroup(getClass().getSimpleName());
   }
 
-  private void addExecutorForVolume(final File volume) {
+  private void addExecutorForVolume(final FsVolumeImpl volume) {
     ThreadFactory threadFactory = new ThreadFactory() {
       int counter = 0;
 
@@ -115,18 +115,21 @@ class FsDatasetAsyncDiskService {
 
     // This can reduce the number of running threads
     executor.allowCoreThreadTimeOut(true);
-    executors.put(volume, executor);
+    executors.put(volume.getStorageID(), executor);
   }
 
   /**
    * Starts AsyncDiskService for a new volume
    * @param volume the root of the new data volume.
    */
-  synchronized void addVolume(File volume) {
+  synchronized void addVolume(FsVolumeImpl volume) {
     if (executors == null) {
       throw new RuntimeException("AsyncDiskService is already shutdown");
     }
-    ThreadPoolExecutor executor = executors.get(volume);
+    if (volume == null) {
+      throw new RuntimeException("Attempt to add a null volume");
+    }
+    ThreadPoolExecutor executor = executors.get(volume.getStorageID());
     if (executor != null) {
       throw new RuntimeException("Volume " + volume + " is already existed.");
     }
@@ -137,17 +140,17 @@ class FsDatasetAsyncDiskService {
    * Stops AsyncDiskService for a volume.
    * @param volume the root of the volume.
    */
-  synchronized void removeVolume(File volume) {
+  synchronized void removeVolume(String storageId) {
     if (executors == null) {
       throw new RuntimeException("AsyncDiskService is already shutdown");
     }
-    ThreadPoolExecutor executor = executors.get(volume);
+    ThreadPoolExecutor executor = executors.get(storageId);
     if (executor == null) {
-      throw new RuntimeException("Can not find volume " + volume
-          + " to remove.");
+      throw new RuntimeException("Can not find volume with storageId "
+          + storageId + " to remove.");
     } else {
       executor.shutdown();
-      executors.remove(volume);
+      executors.remove(storageId);
     }
   }
   
@@ -162,13 +165,16 @@ class FsDatasetAsyncDiskService {
   /**
    * Execute the task sometime in the future, using ThreadPools.
    */
-  synchronized void execute(File root, Runnable task) {
+  synchronized void execute(FsVolumeImpl volume, Runnable task) {
     if (executors == null) {
       throw new RuntimeException("AsyncDiskService is already shutdown");
     }
-    ThreadPoolExecutor executor = executors.get(root);
+    if (volume == null) {
+      throw new RuntimeException("A null volume does not have a executor");
+    }
+    ThreadPoolExecutor executor = executors.get(volume.getStorageID());
     if (executor == null) {
-      throw new RuntimeException("Cannot find root " + root
+      throw new RuntimeException("Cannot find volume " + volume
           + " for execution of task " + task);
     } else {
       executor.execute(task);
@@ -185,7 +191,7 @@ class FsDatasetAsyncDiskService {
     } else {
       LOG.info("Shutting down all async disk service threads");
       
-      for (Map.Entry<File, ThreadPoolExecutor> e : executors.entrySet()) {
+      for (Map.Entry<String, ThreadPoolExecutor> e : executors.entrySet()) {
         e.getValue().shutdown();
       }
       // clear the executor map so that calling execute again will fail.
@@ -198,7 +204,7 @@ class FsDatasetAsyncDiskService {
   public void submitSyncFileRangeRequest(FsVolumeImpl volume,
       final FileDescriptor fd, final long offset, final long nbytes,
       final int flags) {
-    execute(volume.getCurrentDir(), new Runnable() {
+    execute(volume, new Runnable() {
       @Override
       public void run() {
         try {
@@ -220,7 +226,7 @@ class FsDatasetAsyncDiskService {
         + " replica " + replicaToDelete + " for deletion");
     ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
         volumeRef, replicaToDelete, block, trashDirectory);
-    execute(((FsVolumeImpl) volumeRef.getVolume()).getCurrentDir(), deletionTask);
+    execute(((FsVolumeImpl) volumeRef.getVolume()), deletionTask);
   }
   
   /** A task for deleting a block file and its associated meta file, as well

+ 67 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -361,20 +361,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    */
   private static List<VolumeFailureInfo> getInitialVolumeFailureInfos(
       Collection<StorageLocation> dataLocations, DataStorage storage) {
-    Set<String> failedLocationSet = Sets.newHashSetWithExpectedSize(
+    Set<StorageLocation> failedLocationSet = Sets.newHashSetWithExpectedSize(
         dataLocations.size());
     for (StorageLocation sl: dataLocations) {
-      failedLocationSet.add(sl.getFile().getAbsolutePath());
+      LOG.info("Adding to failedLocationSet " + sl);
+      failedLocationSet.add(sl);
     }
     for (Iterator<Storage.StorageDirectory> it = storage.dirIterator();
          it.hasNext(); ) {
       Storage.StorageDirectory sd = it.next();
-      failedLocationSet.remove(sd.getRoot().getAbsolutePath());
+      failedLocationSet.remove(sd.getStorageLocation());
+      LOG.info("Removing from failedLocationSet " + sd.getStorageLocation());
     }
     List<VolumeFailureInfo> volumeFailureInfos = Lists.newArrayListWithCapacity(
         failedLocationSet.size());
     long failureDate = Time.now();
-    for (String failedStorageLocation: failedLocationSet) {
+    for (StorageLocation failedStorageLocation: failedLocationSet) {
       volumeFailureInfos.add(new VolumeFailureInfo(failedStorageLocation,
           failureDate));
     }
@@ -403,49 +405,55 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           new DatanodeStorage(sd.getStorageUuid(),
               DatanodeStorage.State.NORMAL,
               storageType));
-      asyncDiskService.addVolume(sd.getCurrentDir());
+      asyncDiskService.addVolume((FsVolumeImpl) ref.getVolume());
       volumes.addVolume(ref);
     }
   }
 
   private void addVolume(Collection<StorageLocation> dataLocations,
       Storage.StorageDirectory sd) throws IOException {
-    final File dir = sd.getCurrentDir();
-    final StorageType storageType =
-        getStorageTypeFromLocations(dataLocations, sd.getRoot());
+    final StorageLocation storageLocation = sd.getStorageLocation();
 
     // If IOException raises from FsVolumeImpl() or getVolumeMap(), there is
     // nothing needed to be rolled back to make various data structures, e.g.,
     // storageMap and asyncDiskService, consistent.
-    FsVolumeImpl fsVolume = new FsVolumeImpl(
-        this, sd.getStorageUuid(), dir, this.conf, storageType);
+    FsVolumeImpl fsVolume = new FsVolumeImplBuilder()
+                              .setDataset(this)
+                              .setStorageID(sd.getStorageUuid())
+                              .setStorageDirectory(sd)
+                              .setConf(this.conf)
+                              .build();
     FsVolumeReference ref = fsVolume.obtainReference();
     ReplicaMap tempVolumeMap = new ReplicaMap(datasetLock);
     fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker);
 
-    activateVolume(tempVolumeMap, sd, storageType, ref);
-    LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
+    activateVolume(tempVolumeMap, sd, storageLocation.getStorageType(), ref);
+    LOG.info("Added volume - " + storageLocation + ", StorageType: " +
+        storageLocation.getStorageType());
   }
 
   @VisibleForTesting
-  public FsVolumeImpl createFsVolume(String storageUuid, File currentDir,
-      StorageType storageType) throws IOException {
-    return new FsVolumeImpl(this, storageUuid, currentDir, conf, storageType);
+  public FsVolumeImpl createFsVolume(String storageUuid,
+      Storage.StorageDirectory sd,
+      final StorageLocation location) throws IOException {
+    return new FsVolumeImplBuilder()
+        .setDataset(this)
+        .setStorageID(storageUuid)
+        .setStorageDirectory(sd)
+        .setConf(conf)
+        .build();
   }
 
   @Override
   public void addVolume(final StorageLocation location,
       final List<NamespaceInfo> nsInfos)
       throws IOException {
-    final File dir = location.getFile();
-
     // Prepare volume in DataStorage
     final DataStorage.VolumeBuilder builder;
     try {
-      builder = dataStorage.prepareVolume(datanode, location.getFile(), nsInfos);
+      builder = dataStorage.prepareVolume(datanode, location, nsInfos);
     } catch (IOException e) {
-      volumes.addVolumeFailureInfo(new VolumeFailureInfo(
-          location.getFile().getAbsolutePath(), Time.now()));
+      volumes.addVolumeFailureInfo(new VolumeFailureInfo(location, Time.now()));
       throw e;
     }
 
@@ -453,7 +461,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
     StorageType storageType = location.getStorageType();
     final FsVolumeImpl fsVolume =
-        createFsVolume(sd.getStorageUuid(), sd.getCurrentDir(), storageType);
+        createFsVolume(sd.getStorageUuid(), sd, location);
     final ReplicaMap tempVolumeMap = new ReplicaMap(new AutoCloseableLock());
     ArrayList<IOException> exceptions = Lists.newArrayList();
 
@@ -482,34 +490,33 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
     builder.build();
     activateVolume(tempVolumeMap, sd, storageType, ref);
-    LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
+    LOG.info("Added volume - " + location + ", StorageType: " + storageType);
   }
 
   /**
    * Removes a set of volumes from FsDataset.
-   * @param volumesToRemove a set of absolute root path of each volume.
+   * @param storageLocationsToRemove a set of
+   * {@link StorageLocation}s for each volume.
    * @param clearFailure set true to clear failure information.
    */
   @Override
-  public void removeVolumes(Set<File> volumesToRemove, boolean clearFailure) {
-    // Make sure that all volumes are absolute path.
-    for (File vol : volumesToRemove) {
-      Preconditions.checkArgument(vol.isAbsolute(),
-          String.format("%s is not absolute path.", vol.getPath()));
-    }
-
+  public void removeVolumes(
+      Collection<StorageLocation> storageLocationsToRemove,
+      boolean clearFailure) {
     Map<String, List<ReplicaInfo>> blkToInvalidate = new HashMap<>();
     List<String> storageToRemove = new ArrayList<>();
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
         Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
-        final File absRoot = sd.getRoot().getAbsoluteFile();
-        if (volumesToRemove.contains(absRoot)) {
-          LOG.info("Removing " + absRoot + " from FsDataset.");
-
+        final StorageLocation sdLocation = sd.getStorageLocation();
+        LOG.info("Checking removing StorageLocation " +
+            sdLocation + " with id " + sd.getStorageUuid());
+        if (storageLocationsToRemove.contains(sdLocation)) {
+          LOG.info("Removing StorageLocation " + sdLocation + " with id " +
+              sd.getStorageUuid() + " from FsDataset.");
           // Disable the volume from the service.
-          asyncDiskService.removeVolume(sd.getCurrentDir());
-          volumes.removeVolume(absRoot, clearFailure);
+          asyncDiskService.removeVolume(sd.getStorageUuid());
+          volumes.removeVolume(sdLocation, clearFailure);
           volumes.waitVolumeRemoved(5000, datasetLockCondition);
 
           // Removed all replica information for the blocks on the volume.
@@ -517,12 +524,14 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           // not scan disks.
           for (String bpid : volumeMap.getBlockPoolList()) {
             List<ReplicaInfo> blocks = new ArrayList<>();
-            for (Iterator<ReplicaInfo> it = volumeMap.replicas(bpid).iterator();
-                 it.hasNext(); ) {
+            for (Iterator<ReplicaInfo> it =
+                  volumeMap.replicas(bpid).iterator(); it.hasNext();) {
               ReplicaInfo block = it.next();
-              final File absBasePath =
-                  new File(block.getVolume().getBasePath()).getAbsoluteFile();
-              if (absBasePath.equals(absRoot)) {
+              final StorageLocation blockStorageLocation =
+                  block.getVolume().getStorageLocation();
+              LOG.info("checking for block " + block.getBlockId() +
+                  " with storageLocation " + blockStorageLocation);
+              if (blockStorageLocation.equals(sdLocation)) {
                 blocks.add(block);
                 it.remove();
               }
@@ -625,7 +634,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     List<String> failedStorageLocations = Lists.newArrayListWithCapacity(
         infos.length);
     for (VolumeFailureInfo info: infos) {
-      failedStorageLocations.add(info.getFailedStorageLocation());
+      failedStorageLocations.add(
+          info.getFailedStorageLocation().getFile().getAbsolutePath());
     }
     return failedStorageLocations.toArray(
         new String[failedStorageLocations.size()]);
@@ -663,7 +673,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     long lastVolumeFailureDate = 0;
     long estimatedCapacityLostTotal = 0;
     for (VolumeFailureInfo info: infos) {
-      failedStorageLocations.add(info.getFailedStorageLocation());
+      failedStorageLocations.add(
+          info.getFailedStorageLocation().getFile().getAbsolutePath());
       long failureDate = info.getFailureDate();
       if (failureDate > lastVolumeFailureDate) {
         lastVolumeFailureDate = failureDate;
@@ -960,25 +971,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
     FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume();
     // Copy files to temp dir first
-    File[] blockFiles = copyBlockFiles(block.getBlockId(),
-        block.getGenerationStamp(), replicaInfo,
-        targetVolume.getTmpDir(block.getBlockPoolId()),
-        replicaInfo.isOnTransientStorage(), smallBufferSize, conf);
-
-    ReplicaInfo newReplicaInfo = new ReplicaBuilder(ReplicaState.TEMPORARY)
-        .setBlockId(replicaInfo.getBlockId())
-        .setGenerationStamp(replicaInfo.getGenerationStamp())
-        .setFsVolume(targetVolume)
-        .setDirectoryToUse(blockFiles[0].getParentFile())
-        .setBytesToReserve(0)
-        .build();
-    newReplicaInfo.setNumBytes(blockFiles[1].length());
+    ReplicaInfo newReplicaInfo = targetVolume.moveBlockToTmpLocation(block,
+        replicaInfo, smallBufferSize, conf);
+
     // Finalize the copied files
     newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       // Increment numBlocks here as this block moved without knowing to BPS
       FsVolumeImpl volume = (FsVolumeImpl) newReplicaInfo.getVolume();
-      volume.getBlockPoolSlice(block.getBlockPoolId()).incrNumBlocks();
+      volume.incrNumBlocks(block.getBlockPoolId());
     }
 
     removeOldReplica(replicaInfo, newReplicaInfo, block.getBlockPoolId());
@@ -2072,7 +2073,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @return the failed volumes. Returns null if no volume failed.
    */
   @Override // FsDatasetSpi
-  public Set<File> checkDataDir() {
+  public Set<StorageLocation> checkDataDir() {
    return volumes.checkDirs();
   }
     
@@ -2250,9 +2251,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
                     .setFsVolume(vol)
                     .setDirectoryToUse(diskFile.getParentFile())
                     .build();
-              ((FsVolumeImpl) vol).getBlockPoolSlice(bpid)
-                  .resolveDuplicateReplicas(
-                      memBlockInfo, diskBlockInfo, volumeMap);
+              ((FsVolumeImpl) vol).resolveDuplicateReplicas(bpid,
+                  memBlockInfo, diskBlockInfo, volumeMap);
             }
           } else {
             if (!diskFile.delete()) {
@@ -2803,15 +2803,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     // Add thread for DISK volume if RamDisk is configured
     if (ramDiskConfigured &&
         asyncLazyPersistService != null &&
-        !asyncLazyPersistService.queryVolume(v.getCurrentDir())) {
-      asyncLazyPersistService.addVolume(v.getCurrentDir());
+        !asyncLazyPersistService.queryVolume(v)) {
+      asyncLazyPersistService.addVolume(v);
     }
 
     // Remove thread for DISK volume if RamDisk is not configured
     if (!ramDiskConfigured &&
         asyncLazyPersistService != null &&
-        asyncLazyPersistService.queryVolume(v.getCurrentDir())) {
-      asyncLazyPersistService.removeVolume(v.getCurrentDir());
+        asyncLazyPersistService.queryVolume(v)) {
+      asyncLazyPersistService.removeVolume(v);
     }
   }
 
@@ -2946,11 +2946,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
           // Move the replica from lazyPersist/ to finalized/ on
           // the target volume
-          BlockPoolSlice bpSlice =
-              replicaState.getLazyPersistVolume().getBlockPoolSlice(bpid);
-
           newReplicaInfo =
-              bpSlice.activateSavedReplica(replicaInfo, replicaState);
+              replicaState.getLazyPersistVolume().activateSavedReplica(bpid,
+                  replicaInfo, replicaState);
 
           // Update the volumeMap entry.
           volumeMap.add(bpid, newReplicaInfo);

+ 214 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java

@@ -23,11 +23,13 @@ import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
+import java.net.URI;
 import java.nio.channels.ClosedChannelException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.nio.file.StandardCopyOption;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -56,13 +58,18 @@ import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.LocalReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
 import org.apache.hadoop.hdfs.server.datanode.LocalReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.BlockDirFilter;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.CloseableReferenceCount;
@@ -102,8 +109,14 @@ public class FsVolumeImpl implements FsVolumeSpi {
   private final StorageType storageType;
   private final Map<String, BlockPoolSlice> bpSlices
       = new ConcurrentHashMap<String, BlockPoolSlice>();
+
+  // Refers to the base StorageLocation used to construct this volume
+  // (i.e., does not include STORAGE_DIR_CURRENT in
+  // <location>/STORAGE_DIR_CURRENT/)
+  private final StorageLocation storageLocation;
+
   private final File currentDir;    // <StorageDirectory>/current
-  private final DF usage;           
+  private final DF usage;
   private final long reserved;
   private CloseableReferenceCount reference = new CloseableReferenceCount();
 
@@ -124,19 +137,25 @@ public class FsVolumeImpl implements FsVolumeSpi {
    */
   protected ThreadPoolExecutor cacheExecutor;
   
-  FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
-      Configuration conf, StorageType storageType) throws IOException {
+  FsVolumeImpl(FsDatasetImpl dataset, String storageID, StorageDirectory sd,
+      Configuration conf) throws IOException {
+
+    if (sd.getStorageLocation() == null) {
+      throw new IOException("StorageLocation specified for storage directory " +
+          sd + " is null");
+    }
     this.dataset = dataset;
     this.storageID = storageID;
+    this.reservedForReplicas = new AtomicLong(0L);
+    this.storageLocation = sd.getStorageLocation();
+    this.currentDir = sd.getCurrentDir();
+    File parent = currentDir.getParentFile();
+    this.usage = new DF(parent, conf);
+    this.storageType = storageLocation.getStorageType();
     this.reserved = conf.getLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY
         + "." + StringUtils.toLowerCase(storageType.toString()), conf.getLong(
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT));
-    this.reservedForReplicas = new AtomicLong(0L);
-    this.currentDir = currentDir;
-    File parent = currentDir.getParentFile();
-    this.usage = new DF(parent, conf);
-    this.storageType = storageType;
     this.configuredCapacity = -1;
     this.conf = conf;
     cacheExecutor = initializeCacheExecutor(parent);
@@ -285,19 +304,20 @@ public class FsVolumeImpl implements FsVolumeSpi {
     return true;
   }
 
+  @VisibleForTesting
   File getCurrentDir() {
     return currentDir;
   }
   
-  File getRbwDir(String bpid) throws IOException {
+  protected File getRbwDir(String bpid) throws IOException {
     return getBlockPoolSlice(bpid).getRbwDir();
   }
 
-  File getLazyPersistDir(String bpid) throws IOException {
+  protected File getLazyPersistDir(String bpid) throws IOException {
     return getBlockPoolSlice(bpid).getLazypersistDir();
   }
 
-  File getTmpDir(String bpid) throws IOException {
+  protected File getTmpDir(String bpid) throws IOException {
     return getBlockPoolSlice(bpid).getTmpDir();
   }
 
@@ -448,6 +468,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
     return reserved;
   }
 
+  @VisibleForTesting
   BlockPoolSlice getBlockPoolSlice(String bpid) throws IOException {
     BlockPoolSlice bp = bpSlices.get(bpid);
     if (bp == null) {
@@ -457,21 +478,33 @@ public class FsVolumeImpl implements FsVolumeSpi {
   }
 
   @Override
-  public String getBasePath() {
-    return currentDir.getParent();
+  public URI getBaseURI() {
+    return new File(currentDir.getParent()).toURI();
   }
-  
+
   @Override
-  public boolean isTransientStorage() {
-    return storageType.isTransient();
+  public DF getUsageStats(Configuration conf) {
+    if (currentDir != null) {
+      try {
+        return new DF(new File(currentDir.getParent()), conf);
+      } catch (IOException e) {
+        LOG.error("Unable to get disk statistics for volume " + this);
+      }
+    }
+    return null;
   }
 
   @Override
-  public String getPath(String bpid) throws IOException {
-    return getBlockPoolSlice(bpid).getDirectory().getAbsolutePath();
+  public StorageLocation getStorageLocation() {
+    return storageLocation;
   }
 
   @Override
+  public boolean isTransientStorage() {
+    return storageType.isTransient();
+  }
+
+  @VisibleForTesting
   public File getFinalizedDir(String bpid) throws IOException {
     return getBlockPoolSlice(bpid).getFinalizedDir();
   }
@@ -951,7 +984,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
 
   @Override
   public String toString() {
-    return currentDir.getAbsolutePath();
+    return currentDir != null ? currentDir.getParent() : "NULL";
   }
 
   void shutdown() {
@@ -1189,5 +1222,167 @@ public class FsVolumeImpl implements FsVolumeSpi {
         dstBlockFile, true, DFSUtilClient.getSmallBufferSize(conf), conf);
   }
 
+  @Override
+  public LinkedList<ScanInfo> compileReport(String bpid,
+      LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
+      throws InterruptedException, IOException {
+    return compileReport(getFinalizedDir(bpid),
+        getFinalizedDir(bpid), report, reportCompiler);
+  }
+
+  private LinkedList<ScanInfo> compileReport(File bpFinalizedDir,
+      File dir, LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
+        throws InterruptedException {
+
+    reportCompiler.throttle();
+
+    List <String> fileNames;
+    try {
+      fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
+    } catch (IOException ioe) {
+      LOG.warn("Exception occured while compiling report: ", ioe);
+      // Initiate a check on disk failure.
+      dataset.datanode.checkDiskErrorAsync();
+      // Ignore this directory and proceed.
+      return report;
+    }
+    Collections.sort(fileNames);
+
+    /*
+     * Assumption: In the sorted list of files block file appears immediately
+     * before block metadata file. This is true for the current naming
+     * convention for block file blk_<blockid> and meta file
+     * blk_<blockid>_<genstamp>.meta
+     */
+    for (int i = 0; i < fileNames.size(); i++) {
+      // Make sure this thread can make a timely exit. With a low throttle
+      // rate, completing a run can take a looooong time.
+      if (Thread.interrupted()) {
+        throw new InterruptedException();
+      }
+
+      File file = new File(dir, fileNames.get(i));
+      if (file.isDirectory()) {
+        compileReport(bpFinalizedDir, file, report, reportCompiler);
+        continue;
+      }
+      if (!Block.isBlockFilename(file)) {
+        if (isBlockMetaFile(Block.BLOCK_FILE_PREFIX, file.getName())) {
+          long blockId = Block.getBlockId(file.getName());
+          verifyFileLocation(file.getParentFile(), bpFinalizedDir,
+              blockId);
+          report.add(new ScanInfo(blockId, null, file, this));
+        }
+        continue;
+      }
+      File blockFile = file;
+      long blockId = Block.filename2id(file.getName());
+      File metaFile = null;
+
+      // Skip all the files that start with block name until
+      // getting to the metafile for the block
+      while (i + 1 < fileNames.size()) {
+        File blkMetaFile = new File(dir, fileNames.get(i + 1));
+        if (!(blkMetaFile.isFile()
+            && blkMetaFile.getName().startsWith(blockFile.getName()))) {
+          break;
+        }
+        i++;
+        if (isBlockMetaFile(blockFile.getName(), blkMetaFile.getName())) {
+          metaFile = blkMetaFile;
+          break;
+        }
+      }
+      verifyFileLocation(blockFile, bpFinalizedDir, blockId);
+      report.add(new ScanInfo(blockId, blockFile, metaFile, this));
+    }
+    return report;
+  }
+
+  /**
+   * Helper method to determine if a file name is consistent with a block.
+   * meta-data file
+   *
+   * @param blockId the block ID
+   * @param metaFile the file to check
+   * @return whether the file name is a block meta-data file name
+   */
+  private static boolean isBlockMetaFile(String blockId, String metaFile) {
+    return metaFile.startsWith(blockId)
+        && metaFile.endsWith(Block.METADATA_EXTENSION);
+  }
+
+  /**
+   * Verify whether the actual directory location of block file has the
+   * expected directory path computed using its block ID.
+   */
+  private void verifyFileLocation(File actualBlockFile,
+      File bpFinalizedDir, long blockId) {
+    File expectedBlockDir =
+        DatanodeUtil.idToBlockDir(bpFinalizedDir, blockId);
+    File actualBlockDir = actualBlockFile.getParentFile();
+    if (actualBlockDir.compareTo(expectedBlockDir) != 0) {
+      LOG.warn("Block: " + blockId +
+          " found in invalid directory.  Expected directory: " +
+          expectedBlockDir + ".  Actual directory: " + actualBlockDir);
+    }
+  }
+
+  public ReplicaInfo moveBlockToTmpLocation(ExtendedBlock block,
+      ReplicaInfo replicaInfo,
+      int smallBufferSize,
+      Configuration conf) throws IOException {
+
+    File[] blockFiles = FsDatasetImpl.copyBlockFiles(block.getBlockId(),
+        block.getGenerationStamp(), replicaInfo,
+        getTmpDir(block.getBlockPoolId()),
+        replicaInfo.isOnTransientStorage(), smallBufferSize, conf);
+
+    ReplicaInfo newReplicaInfo = new ReplicaBuilder(ReplicaState.TEMPORARY)
+        .setBlockId(replicaInfo.getBlockId())
+        .setGenerationStamp(replicaInfo.getGenerationStamp())
+        .setFsVolume(this)
+        .setDirectoryToUse(blockFiles[0].getParentFile())
+        .setBytesToReserve(0)
+        .build();
+    newReplicaInfo.setNumBytes(blockFiles[1].length());
+    return newReplicaInfo;
+  }
+
+  public File[] copyBlockToLazyPersistLocation(String bpId, long blockId,
+      long genStamp,
+      ReplicaInfo replicaInfo,
+      int smallBufferSize,
+      Configuration conf) throws IOException {
+
+    File lazyPersistDir  = getLazyPersistDir(bpId);
+    if (!lazyPersistDir.exists() && !lazyPersistDir.mkdirs()) {
+      FsDatasetImpl.LOG.warn("LazyWriter failed to create " + lazyPersistDir);
+      throw new IOException("LazyWriter fail to find or " +
+          "create lazy persist dir: " + lazyPersistDir.toString());
+    }
+
+    // No FsDatasetImpl lock for the file copy
+    File[] targetFiles = FsDatasetImpl.copyBlockFiles(
+        blockId, genStamp, replicaInfo, lazyPersistDir, true,
+        smallBufferSize, conf);
+    return targetFiles;
+  }
+
+  public void incrNumBlocks(String bpid) throws IOException {
+    getBlockPoolSlice(bpid).incrNumBlocks();
+  }
+
+  public void resolveDuplicateReplicas(String bpid, ReplicaInfo memBlockInfo,
+      ReplicaInfo diskBlockInfo, ReplicaMap volumeMap) throws IOException {
+    getBlockPoolSlice(bpid).resolveDuplicateReplicas(
+        memBlockInfo, diskBlockInfo, volumeMap);
+  }
+
+  public ReplicaInfo activateSavedReplica(String bpid,
+      ReplicaInfo replicaInfo, RamDiskReplica replicaState) throws IOException {
+    return getBlockPoolSlice(bpid).activateSavedReplica(replicaInfo,
+        replicaState);
+  }
 }
 

+ 65 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+
+/**
+ * This class is to be used as a builder for {@link FsVolumeImpl} objects.
+ */
+public class FsVolumeImplBuilder {
+
+  private FsDatasetImpl dataset;
+  private String storageID;
+  private StorageDirectory sd;
+  private Configuration conf;
+
+  public FsVolumeImplBuilder() {
+    dataset = null;
+    storageID = null;
+    sd = null;
+    conf = null;
+  }
+
+  FsVolumeImplBuilder setDataset(FsDatasetImpl dataset) {
+    this.dataset = dataset;
+    return this;
+  }
+
+  FsVolumeImplBuilder setStorageID(String id) {
+    this.storageID = id;
+    return this;
+  }
+
+  FsVolumeImplBuilder setStorageDirectory(StorageDirectory sd) {
+    this.sd = sd;
+    return this;
+  }
+
+  FsVolumeImplBuilder setConf(Configuration conf) {
+    this.conf = conf;
+    return this;
+  }
+
+  FsVolumeImpl build() throws IOException {
+    return new FsVolumeImpl(dataset, storageID, sd, conf);
+  }
+}

+ 22 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
@@ -41,6 +40,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.AutoCloseableLock;
@@ -51,8 +51,10 @@ class FsVolumeList {
   private final CopyOnWriteArrayList<FsVolumeImpl> volumes =
       new CopyOnWriteArrayList<>();
   // Tracks volume failures, sorted by volume path.
-  private final Map<String, VolumeFailureInfo> volumeFailureInfos =
-      Collections.synchronizedMap(new TreeMap<String, VolumeFailureInfo>());
+  // map from volume storageID to the volume failure info
+  private final Map<StorageLocation, VolumeFailureInfo> volumeFailureInfos =
+      Collections.synchronizedMap(
+          new TreeMap<StorageLocation, VolumeFailureInfo>());
   private final ConcurrentLinkedQueue<FsVolumeImpl> volumesBeingRemoved =
       new ConcurrentLinkedQueue<>();
   private final AutoCloseableLock checkDirsLock;
@@ -234,10 +236,9 @@ class FsVolumeList {
    *
    * @return list of all the failed volumes.
    */
-  Set<File> checkDirs() {
+  Set<StorageLocation> checkDirs() {
     try (AutoCloseableLock lock = checkDirsLock.acquire()) {
-      Set<File> failedVols = null;
-      
+      Set<StorageLocation> failedLocations = null;
       // Make a copy of volumes for performing modification 
       final List<FsVolumeImpl> volumeList = getVolumes();
 
@@ -247,10 +248,10 @@ class FsVolumeList {
           fsv.checkDirs();
         } catch (DiskErrorException e) {
           FsDatasetImpl.LOG.warn("Removing failed volume " + fsv + ": ", e);
-          if (failedVols == null) {
-            failedVols = new HashSet<>(1);
+          if (failedLocations == null) {
+            failedLocations = new HashSet<>(1);
           }
-          failedVols.add(new File(fsv.getBasePath()).getAbsoluteFile());
+          failedLocations.add(fsv.getStorageLocation());
           addVolumeFailureInfo(fsv);
           removeVolume(fsv);
         } catch (ClosedChannelException e) {
@@ -261,13 +262,13 @@ class FsVolumeList {
         }
       }
       
-      if (failedVols != null && failedVols.size() > 0) {
-        FsDatasetImpl.LOG.warn("Completed checkDirs. Found " + failedVols.size()
-            + " failure volumes.");
+      if (failedLocations != null && failedLocations.size() > 0) {
+        FsDatasetImpl.LOG.warn("Completed checkDirs. Found " +
+            failedLocations.size() + " failure volumes.");
       }
 
       waitVolumeRemoved(5000, checkDirsLockCondition);
-      return failedVols;
+      return failedLocations;
     }
   }
 
@@ -315,7 +316,7 @@ class FsVolumeList {
     }
     // If the volume is used to replace a failed volume, it needs to reset the
     // volume failure info for this volume.
-    removeVolumeFailureInfo(new File(volume.getBasePath()));
+    removeVolumeFailureInfo(volume.getStorageLocation());
     FsDatasetImpl.LOG.info("Added new volume: " +
         volume.getStorageID());
   }
@@ -351,16 +352,15 @@ class FsVolumeList {
    * @param volume the volume to be removed.
    * @param clearFailure set true to remove failure info for this volume.
    */
-  void removeVolume(File volume, boolean clearFailure) {
+  void removeVolume(StorageLocation storageLocation, boolean clearFailure) {
     for (FsVolumeImpl fsVolume : volumes) {
-      String basePath = new File(fsVolume.getBasePath()).getAbsolutePath();
-      String targetPath = volume.getAbsolutePath();
-      if (basePath.equals(targetPath)) {
+      StorageLocation baseLocation = fsVolume.getStorageLocation();
+      if (baseLocation.equals(storageLocation)) {
         removeVolume(fsVolume);
       }
     }
     if (clearFailure) {
-      removeVolumeFailureInfo(volume);
+      removeVolumeFailureInfo(storageLocation);
     }
   }
 
@@ -394,13 +394,13 @@ class FsVolumeList {
 
   private void addVolumeFailureInfo(FsVolumeImpl vol) {
     addVolumeFailureInfo(new VolumeFailureInfo(
-        new File(vol.getBasePath()).getAbsolutePath(),
+        vol.getStorageLocation(),
         Time.now(),
         vol.getCapacity()));
   }
 
-  private void removeVolumeFailureInfo(File vol) {
-    volumeFailureInfos.remove(vol.getAbsolutePath());
+  private void removeVolumeFailureInfo(StorageLocation location) {
+    volumeFailureInfos.remove(location);
   }
 
   void addBlockPool(final String bpid, final Configuration conf) throws IOException {

Daži faili netika attēloti, jo izmaiņu fails ir pārāk liels