Sfoglia il codice sorgente

Merge branch 'trunk' into HADOOP-12756

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

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

@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.classification.tools;
+
+import com.sun.javadoc.DocErrorReporter;
+import com.sun.javadoc.LanguageVersion;
+import com.sun.javadoc.RootDoc;
+
+import jdiff.JDiff;
+
+/**
+ * A <a href="http://java.sun.com/javase/6/docs/jdk/api/javadoc/doclet/">Doclet</a>
+ * that only includes class-level elements that are annotated with
+ * {@link org.apache.hadoop.classification.InterfaceAudience.Public}.
+ * Class-level elements with no annotation are excluded.
+ * In addition, all elements that are annotated with
+ * {@link org.apache.hadoop.classification.InterfaceAudience.Private} or
+ * {@link org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate}
+ * are also excluded.
+ * It delegates to the JDiff Doclet, and takes the same options.
+ */
+public class IncludePublicAnnotationsJDiffDoclet {
+
+  public static LanguageVersion languageVersion() {
+    return LanguageVersion.JAVA_1_5;
+  }
+
+  public static boolean start(RootDoc root) {
+    System.out.println(
+        IncludePublicAnnotationsJDiffDoclet.class.getSimpleName());
+    RootDocProcessor.treatUnannotatedClassesAsPrivate = true;
+    return JDiff.start(RootDocProcessor.process(root));
+  }
+
+  public static int optionLength(String option) {
+    Integer length = StabilityOptions.optionLength(option);
+    if (length != null) {
+      return length;
+    }
+    return JDiff.optionLength(option);
+  }
+
+  public static boolean validOptions(String[][] options,
+      DocErrorReporter reporter) {
+    StabilityOptions.validOptions(options, reporter);
+    String[][] filteredOptions = StabilityOptions.filterOptions(options);
+    return JDiff.validOptions(filteredOptions, reporter);
+  }
+}

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

@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
 public abstract class RolloverSignerSecretProvider
 public abstract class RolloverSignerSecretProvider
     extends SignerSecretProvider {
     extends SignerSecretProvider {
 
 
-  private static Logger LOG = LoggerFactory.getLogger(
+  static Logger LOG = LoggerFactory.getLogger(
     RolloverSignerSecretProvider.class);
     RolloverSignerSecretProvider.class);
   /**
   /**
    * Stores the currently valid secrets.  The current secret is the 0th element
    * 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.Properties;
 import java.util.Random;
 import java.util.Random;
 import javax.servlet.ServletContext;
 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.curator.test.TestingServer;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
@@ -25,7 +30,6 @@ import org.junit.Test;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.timeout;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
@@ -34,9 +38,14 @@ public class TestZKSignerSecretProvider {
   private TestingServer zkServer;
   private TestingServer zkServer;
 
 
   // rollover every 2 sec
   // rollover every 2 sec
-  private final int timeout = 4000;
+  private final int timeout = 100;
   private final long rolloverFrequency = timeout / 2;
   private final long rolloverFrequency = timeout / 2;
 
 
+  static final Log LOG = LogFactory.getLog(TestZKSignerSecretProvider.class);
+  {
+    LogManager.getLogger( RolloverSignerSecretProvider.LOG.getName() ).setLevel(Level.DEBUG);
+  }
+
   @Before
   @Before
   public void setup() throws Exception {
   public void setup() throws Exception {
     zkServer = new TestingServer();
     zkServer = new TestingServer();
@@ -60,8 +69,8 @@ public class TestZKSignerSecretProvider {
     byte[] secret2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secret2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secret1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secret1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secret3 = 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();
     Properties config = new Properties();
     config.setProperty(
     config.setProperty(
         ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
         ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
@@ -77,7 +86,8 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertArrayEquals(secret1, allSecrets[0]);
       Assert.assertArrayEquals(secret1, allSecrets[0]);
       Assert.assertNull(allSecrets[1]);
       Assert.assertNull(allSecrets[1]);
-      verify(secretProvider, timeout(timeout).times(1)).rollSecret();
+      verify(secretProvider, timeout(timeout).atLeastOnce()).rollSecret();
+      secretProvider.realRollSecret();
 
 
       currentSecret = secretProvider.getCurrentSecret();
       currentSecret = secretProvider.getCurrentSecret();
       allSecrets = secretProvider.getAllSecrets();
       allSecrets = secretProvider.getAllSecrets();
@@ -85,7 +95,8 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertArrayEquals(secret2, allSecrets[0]);
       Assert.assertArrayEquals(secret2, allSecrets[0]);
       Assert.assertArrayEquals(secret1, allSecrets[1]);
       Assert.assertArrayEquals(secret1, allSecrets[1]);
-      verify(secretProvider, timeout(timeout).times(2)).rollSecret();
+      verify(secretProvider, timeout(timeout).atLeast(2)).rollSecret();
+      secretProvider.realRollSecret();
 
 
       currentSecret = secretProvider.getCurrentSecret();
       currentSecret = secretProvider.getCurrentSecret();
       allSecrets = secretProvider.getAllSecrets();
       allSecrets = secretProvider.getAllSecrets();
@@ -93,128 +104,70 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertEquals(2, allSecrets.length);
       Assert.assertArrayEquals(secret3, allSecrets[0]);
       Assert.assertArrayEquals(secret3, allSecrets[0]);
       Assert.assertArrayEquals(secret2, allSecrets[1]);
       Assert.assertArrayEquals(secret2, allSecrets[1]);
-      verify(secretProvider, timeout(timeout).times(3)).rollSecret();
+      verify(secretProvider, timeout(timeout).atLeast(3)).rollSecret();
+      secretProvider.realRollSecret();
     } finally {
     } finally {
       secretProvider.destroy();
       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
   @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();
     long seedA = System.currentTimeMillis();
     Random rand = new Random(seedA);
     Random rand = new Random(seedA);
     byte[] secretA2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretA2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretA1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretA1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretA3 = 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
     // use the same seed so we can predict the RNG
     long seedB = System.currentTimeMillis() + rand.nextLong();
     long seedB = System.currentTimeMillis() + rand.nextLong();
     rand = new Random(seedB);
     rand = new Random(seedB);
     byte[] secretB2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretB2 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretB1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretB1 = Long.toString(rand.nextLong()).getBytes();
     byte[] secretB3 = 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();
     Properties config = new Properties();
     config.setProperty(
     config.setProperty(
         ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
         ZKSignerSecretProvider.ZOOKEEPER_CONNECTION_STRING,
@@ -223,14 +176,24 @@ public class TestZKSignerSecretProvider {
         "/secret");
         "/secret");
     try {
     try {
       secretProviderA.init(config, getDummyServletContext(), rolloverFrequency);
       secretProviderA.init(config, getDummyServletContext(), rolloverFrequency);
+      secretProviderB.init(config, getDummyServletContext(), rolloverFrequency);
 
 
       byte[] currentSecretA = secretProviderA.getCurrentSecret();
       byte[] currentSecretA = secretProviderA.getCurrentSecret();
       byte[][] allSecretsA = secretProviderA.getAllSecrets();
       byte[][] allSecretsA = secretProviderA.getAllSecrets();
+      byte[] currentSecretB = secretProviderB.getCurrentSecret();
+      byte[][] allSecretsB = secretProviderB.getAllSecrets();
       Assert.assertArrayEquals(secretA1, currentSecretA);
       Assert.assertArrayEquals(secretA1, currentSecretA);
+      Assert.assertArrayEquals(secretA1, currentSecretB);
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertEquals(2, allSecretsA.length);
+      Assert.assertEquals(2, allSecretsB.length);
       Assert.assertArrayEquals(secretA1, allSecretsA[0]);
       Assert.assertArrayEquals(secretA1, allSecretsA[0]);
+      Assert.assertArrayEquals(secretA1, allSecretsB[0]);
       Assert.assertNull(allSecretsA[1]);
       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();
       currentSecretA = secretProviderA.getCurrentSecret();
       allSecretsA = secretProviderA.getAllSecrets();
       allSecretsA = secretProviderA.getAllSecrets();
@@ -238,18 +201,32 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertArrayEquals(secretA2, allSecretsA[0]);
       Assert.assertArrayEquals(secretA2, allSecretsA[0]);
       Assert.assertArrayEquals(secretA1, allSecretsA[1]);
       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.assertArrayEquals(secretA2, currentSecretB);
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertEquals(2, allSecretsA.length);
       Assert.assertArrayEquals(secretA2, allSecretsB[0]);
       Assert.assertArrayEquals(secretA2, allSecretsB[0]);
       Assert.assertArrayEquals(secretA1, allSecretsB[1]);
       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();
       currentSecretA = secretProviderA.getCurrentSecret();
       allSecretsA = secretProviderA.getAllSecrets();
       allSecretsA = secretProviderA.getAllSecrets();
@@ -260,13 +237,13 @@ public class TestZKSignerSecretProvider {
       Assert.assertEquals(2, allSecretsB.length);
       Assert.assertEquals(2, allSecretsB.length);
       Assert.assertArrayEquals(allSecretsA[0], allSecretsB[0]);
       Assert.assertArrayEquals(allSecretsA[0], allSecretsB[0]);
       Assert.assertArrayEquals(allSecretsA[1], allSecretsB[1]);
       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 {
     } finally {
       secretProviderB.destroy();
       secretProviderB.destroy();

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

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

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


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

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

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

@@ -103,8 +103,9 @@ import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXException;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 
 
-/** 
+/**
  * Provides access to configuration parameters.
  * Provides access to configuration parameters.
  *
  *
  * <h4 id="Resources">Resources</h4>
  * <h4 id="Resources">Resources</h4>
@@ -2834,14 +2835,37 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     writeXml(new OutputStreamWriter(out, "UTF-8"));
     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.
    * @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 {
     try {
       DOMSource source = new DOMSource(doc);
       DOMSource source = new DOMSource(doc);
@@ -2861,62 +2885,180 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   /**
   /**
    * Return the XML DOM corresponding to this Configuration.
    * 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;
     Document doc;
     try {
     try {
-      doc =
-        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
+      doc = DocumentBuilderFactory
+          .newInstance()
+          .newDocumentBuilder()
+          .newDocument();
     } catch (ParserConfigurationException pe) {
     } catch (ParserConfigurationException pe) {
       throw new IOException(pe);
       throw new IOException(pe);
     }
     }
+
     Element conf = doc.createElement("configuration");
     Element conf = doc.createElement("configuration");
     doc.appendChild(conf);
     doc.appendChild(conf);
     conf.appendChild(doc.createTextNode("\n"));
     conf.appendChild(doc.createTextNode("\n"));
     handleDeprecation(); //ensure properties is set and deprecation is handled
     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
    * @param out the Writer to write to
    * @throws IOException
    * @throws IOException
    */
    */
@@ -2930,29 +3072,47 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     dumpGenerator.flush();
     dumpGenerator.flush();
     synchronized (config) {
     synchronized (config) {
       for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
       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.writeEndArray();
     dumpGenerator.writeEndObject();
     dumpGenerator.writeEndObject();
     dumpGenerator.flush();
     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.
    * Get the {@link ClassLoader} for this job.
-   * 
+   *
    * @return the correct class loader.
    * @return the correct class loader.
    */
    */
   public ClassLoader getClassLoader() {
   public ClassLoader getClassLoader() {

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

@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Fast but inaccurate class to tell how much space HDFS is using.
+ * This class makes the assumption that the entire mount is used for
+ * HDFS and that no two hdfs data dirs are on the same disk.
+ *
+ * To use set fs.getspaceused.classname
+ * to org.apache.hadoop.fs.DFCachingGetSpaceUsed in your core-site.xml
+ *
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class DFCachingGetSpaceUsed extends CachingGetSpaceUsed {
+  private final DF df;
+
+  public DFCachingGetSpaceUsed(Builder builder) throws IOException {
+    super(builder);
+    this.df = new DF(builder.getPath(), builder.getInterval());
+  }
+
+  @Override
+  protected void refresh() {
+    this.used.set(df.getUsed());
+  }
+}

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

@@ -31,12 +31,13 @@ import java.io.IOException;
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class DU extends CachingGetSpaceUsed {
 public class DU extends CachingGetSpaceUsed {
-  private DUShell duShell;
+  private final DUShell duShell;
 
 
   @VisibleForTesting
   @VisibleForTesting
-  public DU(File path, long interval, long jitter, long initialUsed)
+   public DU(File path, long interval, long jitter, long initialUsed)
       throws IOException {
       throws IOException {
     super(path, interval, jitter, initialUsed);
     super(path, interval, jitter, initialUsed);
+    this.duShell = new DUShell();
   }
   }
 
 
   public DU(CachingGetSpaceUsed.Builder builder) throws IOException {
   public DU(CachingGetSpaceUsed.Builder builder) throws IOException {
@@ -48,9 +49,6 @@ public class DU extends CachingGetSpaceUsed {
 
 
   @Override
   @Override
   protected synchronized void refresh() {
   protected synchronized void refresh() {
-    if (duShell == null) {
-      duShell = new DUShell();
-    }
     try {
     try {
       duShell.startRefresh();
       duShell.startRefresh();
     } catch (IOException ioe) {
     } 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("}");
     builder.append("}");
     return builder.toString();
     return builder.toString();
   }
   }
+
+  /**
+   * A frozen version of {@link #toString()} to be backward compatible.
+   * When backward compatibility is not needed, use {@link #toString()}, which
+   * provides more info and is supposed to evolve.
+   * Don't change this method except for major revisions.
+   *
+   * NOTE:
+   * Currently this method is used by CLI for backward compatibility.
+   */
+  public String toStringStable() {
+    StringBuilder builder = new StringBuilder("{");
+    builder.append("cipherSuite: " + cipherSuite);
+    builder.append(", cryptoProtocolVersion: " + version);
+    builder.append(", edek: " + Hex.encodeHexString(edek));
+    builder.append(", iv: " + Hex.encodeHexString(iv));
+    builder.append(", keyName: " + keyName);
+    builder.append(", ezKeyVersionName: " + ezKeyVersionName);
+    builder.append("}");
+    return builder.toString();
+  }
 }
 }

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

@@ -2858,7 +2858,15 @@ public abstract class FileSystem extends Configured implements Closeable {
                   ClassUtil.findContainingJar(fs.getClass()), e);
                   ClassUtil.findContainingJar(fs.getClass()), e);
             }
             }
           } catch (ServiceConfigurationError ee) {
           } 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;
         FILE_SYSTEMS_LOADED = true;
@@ -2944,7 +2952,8 @@ public abstract class FileSystem extends Configured implements Closeable {
         }
         }
         fs.key = key;
         fs.key = key;
         map.put(key, fs);
         map.put(key, fs);
-        if (conf.getBoolean("fs.automatic.close", true)) {
+        if (conf.getBoolean(
+            FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
           toAutoClose.add(key);
           toAutoClose.add(key);
         }
         }
         return fs;
         return fs;

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

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

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

@@ -36,7 +36,7 @@ import org.apache.hadoop.util.StringUtils;
  * to create a new instance.
  * to create a new instance.
  */
  */
 @InterfaceAudience.Public
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public class AclEntry {
 public class AclEntry {
   private final AclEntryType type;
   private final AclEntryType type;
   private final String name;
   private final String name;
@@ -100,13 +100,29 @@ public class AclEntry {
   }
   }
 
 
   @Override
   @Override
+  @InterfaceStability.Unstable
   public String toString() {
   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();
     StringBuilder sb = new StringBuilder();
     if (scope == AclEntryScope.DEFAULT) {
     if (scope == AclEntryScope.DEFAULT) {
       sb.append("default:");
       sb.append("default:");
     }
     }
     if (type != null) {
     if (type != null) {
-      sb.append(StringUtils.toLowerCase(type.toString()));
+      sb.append(StringUtils.toLowerCase(type.toStringStable()));
     }
     }
     sb.append(':');
     sb.append(':');
     if (name != null) {
     if (name != null) {
@@ -203,6 +219,8 @@ public class AclEntry {
   /**
   /**
    * Parses a string representation of an ACL spec into a list of AclEntry
    * Parses a string representation of an ACL spec into a list of AclEntry
    * objects. Example: "user::rwx,user:foo:rw-,group::r--,other::---"
    * 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
    * @param aclSpec
    *          String representation of an ACL spec.
    *          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>
    * 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
    * @param aclStr
    *          String representation of an ACL.<br>
    *          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.
  * Specifies the scope or intended usage of an ACL entry.
  */
  */
 @InterfaceAudience.Public
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public enum AclEntryScope {
 public enum AclEntryScope {
   /**
   /**
    * An ACL entry that is inspected during permission checks to enforce
    * 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.
  * Specifies the type of an ACL entry.
  */
  */
 @InterfaceAudience.Public
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public enum AclEntryType {
 public enum AclEntryType {
   /**
   /**
    * An ACL entry applied to a specific user.  These ACL entries can be unnamed,
    * 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.
    * of the more specific ACL entry types.
    */
    */
   OTHER;
   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.
  * instances are immutable. Use a {@link Builder} to create a new instance.
  */
  */
 @InterfaceAudience.Public
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public class AclStatus {
 public class AclStatus {
   private final String owner;
   private final String owner;
   private final String group;
   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)) {
       if (AclUtil.isMinimalAcl(entries)) {
         for (AclEntry entry: entries) {
         for (AclEntry entry: entries) {
-          out.println(entry);
+          out.println(entry.toStringStable());
         }
         }
       } else {
       } else {
         for (AclEntry entry: entries) {
         for (AclEntry entry: entries) {
@@ -145,10 +145,10 @@ class AclCommands extends FsCommand {
           out.println(String.format("%s\t#effective:%s", entry,
           out.println(String.format("%s\t#effective:%s", entry,
             effectivePerm.SYMBOL));
             effectivePerm.SYMBOL));
         } else {
         } else {
-          out.println(entry);
+          out.println(entry.toStringStable());
         }
         }
       } else {
       } else {
-        out.println(entry);
+        out.println(entry.toStringStable());
       }
       }
     }
     }
   }
   }

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ipc;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.ipc.Server.Call;
+import org.apache.hadoop.security.UserGroupInformation;
+
+public abstract class ExternalCall<T> extends Call {
+  private final PrivilegedExceptionAction<T> action;
+  private final AtomicBoolean done = new AtomicBoolean();
+  private T result;
+  private Throwable error;
+
+  public ExternalCall(PrivilegedExceptionAction<T> action) {
+    this.action = action;
+  }
+
+  public abstract UserGroupInformation getRemoteUser();
+
+  public final T get() throws IOException, InterruptedException {
+    waitForCompletion();
+    if (error != null) {
+      if (error instanceof IOException) {
+        throw (IOException)error;
+      } else {
+        throw new IOException(error);
+      }
+    }
+    return result;
+  }
+
+  // wait for response to be triggered to support postponed calls
+  private void waitForCompletion() throws InterruptedException {
+    synchronized(done) {
+      while (!done.get()) {
+        try {
+          done.wait();
+        } catch (InterruptedException ie) {
+          if (Thread.interrupted()) {
+            throw ie;
+          }
+        }
+      }
+    }
+  }
+
+  boolean isDone() {
+    return done.get();
+  }
+
+  // invoked by ipc handler
+  @Override
+  public final Void run() throws IOException {
+    try {
+      result = action.run();
+      sendResponse();
+    } catch (Throwable t) {
+      abortResponse(t);
+    }
+    return null;
+  }
+
+  @Override
+  final void doResponse(Throwable t) {
+    synchronized(done) {
+      error = t;
+      done.set(true);
+      done.notify();
+    }
+  }
+}

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

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

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

@@ -813,7 +813,7 @@ public class NetworkTopology {
       }
       }
     }
     }
     if (numOfDatanodes == 0) {
     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));
           String.valueOf(scope), String.valueOf(excludedScope));
       return null;
       return null;
     }
     }

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

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

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

@@ -104,12 +104,8 @@ public class Credentials implements Writable {
       for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
       for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
           tokenMap.entrySet()) {
           tokenMap.entrySet()) {
         Token<? extends TokenIdentifier> token = e.getValue();
         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);
       tokenMap.putAll(tokensToAdd);

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

@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.security;
+
+import static org.apache.hadoop.security.UGIExceptionMessages.*;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when {@link UserGroupInformation} failed with an unrecoverable error,
+ * such as failure in kerberos login/logout, invalid subject etc.
+ *
+ * Caller should not retry when catching this exception.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class KerberosAuthException extends IOException {
+  static final long serialVersionUID = 31L;
+
+  private String user;
+  private String principal;
+  private String keytabFile;
+  private String ticketCacheFile;
+  private String initialMessage;
+
+  public KerberosAuthException(String msg) {
+    super(msg);
+  }
+
+  public KerberosAuthException(Throwable cause) {
+    super(cause);
+  }
+
+  public KerberosAuthException(String initialMsg, Throwable cause) {
+    this(cause);
+    initialMessage = initialMsg;
+  }
+
+  public void setUser(final String u) {
+    user = u;
+  }
+
+  public void setPrincipal(final String p) {
+    principal = p;
+  }
+
+  public void setKeytabFile(final String k) {
+    keytabFile = k;
+  }
+
+  public void setTicketCacheFile(final String t) {
+    ticketCacheFile = t;
+  }
+
+  /** @return The initial message, or null if not set. */
+  public String getInitialMessage() {
+    return initialMessage;
+  }
+
+  /** @return The keytab file path, or null if not set. */
+  public String getKeytabFile() {
+    return keytabFile;
+  }
+
+  /** @return The principal, or null if not set. */
+  public String getPrincipal() {
+    return principal;
+  }
+
+  /** @return The ticket cache file path, or null if not set. */
+  public String getTicketCacheFile() {
+    return ticketCacheFile;
+  }
+
+  /** @return The user, or null if not set. */
+  public String getUser() {
+    return user;
+  }
+
+  @Override
+  public String getMessage() {
+    final StringBuilder sb = new StringBuilder();
+    if (initialMessage != null) {
+      sb.append(initialMessage);
+    }
+    if (user != null) {
+      sb.append(FOR_USER + user);
+    }
+    if (principal != null) {
+      sb.append(FOR_PRINCIPAL + principal);
+    }
+    if (keytabFile != null) {
+      sb.append(FROM_KEYTAB + keytabFile);
+    }
+    if (ticketCacheFile != null) {
+      sb.append(USING_TICKET_CACHE_FILE+ ticketCacheFile);
+    }
+    sb.append(" " + super.getMessage());
+    return sb.toString();
+  }
+}

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

@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.security;
+
+/**
+ * Standard strings to use in exception messages
+ * in {@link KerberosAuthException} when throwing.
+ */
+final class UGIExceptionMessages {
+
+  public static final String FAILURE_TO_LOGIN = "failure to login:";
+  public static final String FOR_USER = " for user: ";
+  public static final String FOR_PRINCIPAL = " for principal: ";
+  public static final String FROM_KEYTAB = " from keytab ";
+  public static final String LOGIN_FAILURE = "Login failure";
+  public static final String LOGOUT_FAILURE = "Logout failure";
+  public static final String MUST_FIRST_LOGIN =
+      "login must be done first";
+  public static final String MUST_FIRST_LOGIN_FROM_KEYTAB =
+      "loginUserFromKeyTab must be done first";
+  public static final String SUBJECT_MUST_CONTAIN_PRINCIPAL =
+      "Provided Subject must contain a KerberosPrincipal";
+  public static final String SUBJECT_MUST_NOT_BE_NULL =
+      "Subject must not be null";
+  public static final String USING_TICKET_CACHE_FILE =
+      " using ticket cache file: ";
+
+  //checkstyle: Utility classes should not have a public or default constructor.
+  private UGIExceptionMessages() {
+  }
+}

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

@@ -21,6 +21,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_MET
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
 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.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
+import static org.apache.hadoop.security.UGIExceptionMessages.*;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 
 import java.io.File;
 import java.io.File;
@@ -38,7 +39,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
@@ -652,33 +652,7 @@ public class UserGroupInformation {
     }
     }
     this.isKrbTkt = KerberosUtil.hasKerberosTicket(subject);
     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
    * checks if logged in using kerberos
    * @return true if the subject logged via keytab or has a Kerberos TGT
    * @return true if the subject logged via keytab or has a Kerberos TGT
@@ -782,8 +756,11 @@ public class UserGroupInformation {
       ugi.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
       ugi.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
       return ugi;
       return ugi;
     } catch (LoginException le) {
     } 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
    * @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)
   public static UserGroupInformation getUGIFromSubject(Subject subject)
       throws IOException {
       throws IOException {
     if (subject == null) {
     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()) {
     if (subject.getPrincipals(KerberosPrincipal.class).isEmpty()) {
-      throw new IOException("Provided Subject must contain a KerberosPrincipal");
+      throw new KerberosAuthException(SUBJECT_MUST_CONTAIN_PRINCIPAL);
     }
     }
 
 
     KerberosPrincipal principal =
     KerberosPrincipal principal =
@@ -921,7 +899,7 @@ public class UserGroupInformation {
       loginUser.spawnAutoRenewalThreadForUserCreds();
       loginUser.spawnAutoRenewalThreadForUserCreds();
     } catch (LoginException le) {
     } catch (LoginException le) {
       LOG.debug("failure to login", 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()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("UGI loginUser:"+loginUser);
       LOG.debug("UGI loginUser:"+loginUser);
@@ -1028,7 +1006,8 @@ public class UserGroupInformation {
    * file and logs them in. They become the currently logged-in user.
    * file and logs them in. They become the currently logged-in user.
    * @param user the principal name to load from the keytab
    * @param user the principal name to load from the keytab
    * @param path the path to the keytab file
    * @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
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   @InterfaceStability.Evolving
@@ -1057,8 +1036,10 @@ public class UserGroupInformation {
       if (start > 0) {
       if (start > 0) {
         metrics.loginFailure.add(Time.now() - start);
         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
     LOG.info("Login successful for user " + keytabPrincipal
         + " using keytab file " + keytabFile);
         + " using keytab file " + keytabFile);
@@ -1069,8 +1050,9 @@ public class UserGroupInformation {
    * This method assumes that the user logged in by calling
    * This method assumes that the user logged in by calling
    * {@link #loginUserFromKeytab(String, String)}.
    * {@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
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   @InterfaceStability.Evolving
@@ -1081,7 +1063,7 @@ public class UserGroupInformation {
     }
     }
     LoginContext login = getLogin();
     LoginContext login = getLogin();
     if (login == null || keytabFile == null) {
     if (login == null || keytabFile == null) {
-      throw new IOException("loginUserFromKeytab must be done first");
+      throw new KerberosAuthException(MUST_FIRST_LOGIN_FROM_KEYTAB);
     }
     }
 
 
     try {
     try {
@@ -1092,9 +1074,10 @@ public class UserGroupInformation {
         login.logout();
         login.logout();
       }
       }
     } catch (LoginException le) {
     } 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
     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.
    * Re-login a user from keytab if TGT is expired or is close to expiry.
    * 
    * 
    * @throws IOException
    * @throws IOException
+   * @throws KerberosAuthException if it's a kerberos login exception.
    */
    */
   public synchronized void checkTGTAndReloginFromKeytab() throws IOException {
   public synchronized void checkTGTAndReloginFromKeytab() throws IOException {
     if (!isSecurityEnabled()
     if (!isSecurityEnabled()
@@ -1126,12 +1110,12 @@ public class UserGroupInformation {
    * happened already.
    * happened already.
    * The Subject field of this UserGroupInformation object is updated to have
    * The Subject field of this UserGroupInformation object is updated to have
    * the new credentials.
    * the new credentials.
-   * @throws IOException on a failure
+   * @throws IOException
+   * @throws KerberosAuthException on a failure
    */
    */
   @InterfaceAudience.Public
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   @InterfaceStability.Evolving
-  public synchronized void reloginFromKeytab()
-  throws IOException {
+  public synchronized void reloginFromKeytab() throws IOException {
     if (!isSecurityEnabled() ||
     if (!isSecurityEnabled() ||
          user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
          user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
          !isKeytab)
          !isKeytab)
@@ -1151,7 +1135,7 @@ public class UserGroupInformation {
     
     
     LoginContext login = getLogin();
     LoginContext login = getLogin();
     if (login == null || keytabFile == null) {
     if (login == null || keytabFile == null) {
-      throw new IOException("loginUserFromKeyTab must be done first");
+      throw new KerberosAuthException(MUST_FIRST_LOGIN_FROM_KEYTAB);
     }
     }
     
     
     long start = 0;
     long start = 0;
@@ -1183,8 +1167,10 @@ public class UserGroupInformation {
       if (start > 0) {
       if (start > 0) {
         metrics.loginFailure.add(Time.now() - start);
         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.
    * method assumes that login had happened already.
    * The Subject field of this UserGroupInformation object is updated to have
    * The Subject field of this UserGroupInformation object is updated to have
    * the new credentials.
    * the new credentials.
-   * @throws IOException on a failure
+   * @throws IOException
+   * @throws KerberosAuthException on a failure
    */
    */
   @InterfaceAudience.Public
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   @InterfaceStability.Evolving
-  public synchronized void reloginFromTicketCache()
-  throws IOException {
+  public synchronized void reloginFromTicketCache() throws IOException {
     if (!isSecurityEnabled() || 
     if (!isSecurityEnabled() || 
         user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
         user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS ||
         !isKrbTkt)
         !isKrbTkt)
       return;
       return;
     LoginContext login = getLogin();
     LoginContext login = getLogin();
     if (login == null) {
     if (login == null) {
-      throw new IOException("login must be done first");
+      throw new KerberosAuthException(MUST_FIRST_LOGIN);
     }
     }
     long now = Time.now();
     long now = Time.now();
     if (!hasSufficientTimeElapsed(now)) {
     if (!hasSufficientTimeElapsed(now)) {
@@ -1232,8 +1218,9 @@ public class UserGroupInformation {
       login.login();
       login.login();
       setLogin(login);
       setLogin(login);
     } catch (LoginException le) {
     } 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) {
       if (start > 0) {
         metrics.loginFailure.add(Time.now() - start);
         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 {
     } finally {
       if(oldKeytabFile != null) keytabFile = oldKeytabFile;
       if(oldKeytabFile != null) keytabFile = oldKeytabFile;
       if(oldKeytabPrincipal != null) keytabPrincipal = oldKeytabPrincipal;
       if(oldKeytabPrincipal != null) keytabPrincipal = oldKeytabPrincipal;
@@ -1611,7 +1600,7 @@ public class UserGroupInformation {
       Credentials creds = new Credentials(getCredentialsInternal());
       Credentials creds = new Credentials(getCredentialsInternal());
       Iterator<Token<?>> iter = creds.getAllTokens().iterator();
       Iterator<Token<?>> iter = creds.getAllTokens().iterator();
       while (iter.hasNext()) {
       while (iter.hasNext()) {
-        if (iter.next() instanceof Token.PrivateToken) {
+        if (iter.next().isPrivate()) {
           iter.remove();
           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;
     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
    * Indicates whether the token is a clone.  Used by HA failover proxy
    * to indicate a token should not be visible to the user via
    * to indicate a token should not be visible to the user via
    * UGI.getCredentials()
    * 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;
     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
     @Override

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

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

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

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

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

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

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

@@ -93,7 +93,7 @@
   <description>
   <description>
     The name of the Network Interface from which the service should determine
     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,
     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
     Kerberos principal. If this configuration value is not set, the service
     will use its default hostname as returned by
     will use its default hostname as returned by
     InetAddress.getLocalHost().getCanonicalHostName().
     InetAddress.getLocalHost().getCanonicalHostName().
@@ -400,7 +400,7 @@
     The number of levels to go up the group hierarchy when determining
     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
     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
     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.
     hadoop.security.group.mapping.ldap.directory.search.timeout.
     The default will usually be appropriate for all LDAP systems.
     The default will usually be appropriate for all LDAP systems.
   </description>
   </description>
@@ -1939,7 +1939,7 @@
   <value>dr.who=;</value>
   <value>dr.who=;</value>
   <description>
   <description>
     Static mapping of user to groups. This will override the groups if
     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
     look-up will not happen for these users, instead groups mapped in this
     configuration will be used.
     configuration will be used.
     Mapping should be in this format.
     Mapping should be in this format.

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

@@ -504,7 +504,7 @@ See [HDFS Snapshots Guide](../hadoop-hdfs/HdfsSnapshots.html).
 rm
 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.
 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 deletes the directory and any content under it recursively.
 * The -r option is equivalent to -R.
 * 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 -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:
 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)`
 ### `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
 #### 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
     if isDir(FS, p) and not recursive and (children(FS, p) != {}) : raise IOException
 
 
+(HDFS raises `PathIsNotEmptyDirectoryException` here.)
 
 
 #### Postconditions
 #### Postconditions
 
 
 
 
 ##### Nonexistent path
 ##### 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):
     if not exists(FS, p):
         FS' = FS
         FS' = FS
@@ -700,7 +721,7 @@ A path referring to a file is removed, return value: `True`
         result = True
         result = True
 
 
 
 
-##### Empty root directory
+##### Empty root directory, `recursive == False`
 
 
 Deleting an empty root does not change the filesystem state
 Deleting an empty root does not change the filesystem state
 and may return true or false.
 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.
 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
 Deleting an empty directory that is not root will remove the path from the FS and
 return true.
 return true.
@@ -721,26 +745,41 @@ return true.
         result = True
         result = True
 
 
 
 
-##### Recursive delete of root directory
+##### Recursive delete of non-empty root directory
 
 
 Deleting a root path with children and `recursive==True`
 Deleting a root path with children and `recursive==True`
  can do one of two things.
  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,
 the correct permissions to delete everything,
 they are free to do so (resulting in an empty filesystem).
 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.
 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
 ##### Recursive delete of non-root directory
 
 
@@ -766,11 +805,11 @@ removes the path and all descendants
 
 
 #### Implementation Notes
 #### 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)`
 ### `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;
 package org.apache.hadoop.conf;
 
 
 import java.io.StringWriter;
 import java.io.StringWriter;
+import java.io.PrintWriter;
 import java.io.StringReader;
 import java.io.StringReader;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 
 
 import javax.servlet.http.HttpServletRequest;
 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.ws.rs.core.HttpHeaders;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -34,17 +38,36 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 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.junit.Test;
 import org.mockito.Mockito;
 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
  * Basic test case that the ConfServlet can write configuration
  * to its output in XML and JSON format.
  * 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_KEY = "testconfservlet.key";
   private static final String TEST_VAL = "testval";
   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() {
   private Configuration getTestConf() {
     Configuration testConf = new Configuration();
     Configuration testConf = new Configuration();
@@ -52,6 +75,14 @@ public class TestConfServlet extends TestCase {
     return testConf;
     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
   @Test
   public void testParseHeaders() throws Exception {
   public void testParseHeaders() throws Exception {
     HashMap<String, String> verifyMap = new HashMap<String, String>();
     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
   @Test
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public void testWriteJson() throws Exception {
   public void testWriteJson() throws Exception {
@@ -109,7 +226,6 @@ public class TestConfServlet extends TestCase {
     for (int i = 0; i < nameNodes.getLength(); i++) {
     for (int i = 0; i < nameNodes.getLength(); i++) {
       Node nameNode = nameNodes.item(i);
       Node nameNode = nameNodes.item(i);
       String key = nameNode.getTextContent();
       String key = nameNode.getTextContent();
-      System.err.println("xml key: " + key);
       if (TEST_KEY.equals(key)) {
       if (TEST_KEY.equals(key)) {
         foundSetting = true;
         foundSetting = true;
         Element propertyElem = (Element)nameNode.getParentNode();
         Element propertyElem = (Element)nameNode.getParentNode();

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

@@ -42,7 +42,6 @@ import static java.util.concurrent.TimeUnit.*;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.fail;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
@@ -1140,7 +1139,19 @@ public class TestConfiguration extends TestCase {
       this.properties = properties;
       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 {
   static class JsonProperty {
     String key;
     String key;
     public String getKey() {
     public String getKey() {
@@ -1171,7 +1182,14 @@ public class TestConfiguration extends TestCase {
     boolean isFinal;
     boolean isFinal;
     String resource;
     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 {
   public void testGetSetTrimmedNames() throws IOException {
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.set(" name", "value");
     conf.set(" name", "value");
@@ -1180,7 +1198,121 @@ public class TestConfiguration extends TestCase {
     assertEquals("value", conf.getRaw("  name  "));
     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();
     StringWriter outWriter = new StringWriter();
     Configuration.dumpConfiguration(conf, outWriter);
     Configuration.dumpConfiguration(conf, outWriter);
     String jsonStr = outWriter.toString();
     String jsonStr = outWriter.toString();

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

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

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

@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test to make sure df can run and work.
+ */
+public class TestDFCachingGetSpaceUsed {
+  final static private File DF_DIR = GenericTestUtils.getTestDir("testdfspace");
+  public static final int FILE_SIZE = 1024;
+
+  @Before
+  public void setUp() {
+    FileUtil.fullyDelete(DF_DIR);
+    assertTrue(DF_DIR.mkdirs());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    FileUtil.fullyDelete(DF_DIR);
+  }
+
+  @Test
+  public void testCanBuildRun() throws Exception {
+    File file = writeFile("testCanBuild");
+
+    GetSpaceUsed instance = new CachingGetSpaceUsed.Builder()
+        .setPath(file)
+        .setInterval(50060)
+        .setKlass(DFCachingGetSpaceUsed.class)
+        .build();
+    assertTrue(instance instanceof DFCachingGetSpaceUsed);
+    assertTrue(instance.getUsed() >= FILE_SIZE - 20);
+    ((DFCachingGetSpaceUsed) instance).close();
+  }
+
+  private File writeFile(String fileName) throws IOException {
+    File f = new File(DF_DIR, fileName);
+    assertTrue(f.createNewFile());
+    RandomAccessFile randomAccessFile = new RandomAccessFile(f, "rws");
+    randomAccessFile.writeUTF(RandomStringUtils.randomAlphabetic(FILE_SIZE));
+    randomAccessFile.getFD().sync();
+    randomAccessFile.close();
+    return f;
+  }
+
+}

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

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

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

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

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

@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.FileStatus;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 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.toList;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk;
 
 
@@ -62,12 +64,40 @@ public abstract class AbstractContractRootDirectoryTest extends AbstractFSContra
   }
   }
 
 
   @Test
   @Test
-  public void testRmEmptyRootDirNonRecursive() throws Throwable {
+  public void testRmEmptyRootDirRecursive() throws Throwable {
     //extra sanity checks here to avoid support calls about complete loss of data
     //extra sanity checks here to avoid support calls about complete loss of data
     skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
     skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
     Path root = new Path("/");
     Path root = new Path("/");
     assertIsDirectory(root);
     assertIsDirectory(root);
     boolean deleted = getFileSystem().delete(root, true);
     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);
     LOG.info("rm / of empty dir result is {}", deleted);
     assertIsDirectory(root);
     assertIsDirectory(root);
   }
   }
@@ -88,6 +118,8 @@ public abstract class AbstractContractRootDirectoryTest extends AbstractFSContra
     } catch (IOException e) {
     } catch (IOException e) {
       //expected
       //expected
       handleExpectedException(e);
       handleExpectedException(e);
+      // and the file must still be present
+      assertIsFile(file);
     } finally {
     } finally {
       getFileSystem().delete(file, false);
       getFileSystem().delete(file, false);
     }
     }

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

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

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

@@ -47,6 +47,9 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.Set;
 import java.util.UUID;
 import java.util.UUID;
 
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
 /**
  * Utilities used across test cases.
  * Utilities used across test cases.
  */
  */
@@ -55,8 +58,6 @@ public class ContractTestUtils extends Assert {
   private static final Logger LOG =
   private static final Logger LOG =
       LoggerFactory.getLogger(ContractTestUtils.class);
       LoggerFactory.getLogger(ContractTestUtils.class);
 
 
-  public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size";
-
   // For scale testing, we can repeatedly write small chunk data to generate
   // For scale testing, we can repeatedly write small chunk data to generate
   // a large file.
   // a large file.
   public static final String IO_CHUNK_BUFFER_SIZE = "io.chunk.buffer.size";
   public static final String IO_CHUNK_BUFFER_SIZE = "io.chunk.buffer.size";
@@ -150,8 +151,8 @@ public class ContractTestUtils extends Assert {
     FSDataOutputStream out = fs.create(path,
     FSDataOutputStream out = fs.create(path,
                                        overwrite,
                                        overwrite,
                                        fs.getConf()
                                        fs.getConf()
-                                         .getInt(IO_FILE_BUFFER_SIZE,
-                                                 4096),
+                                         .getInt(IO_FILE_BUFFER_SIZE_KEY,
+                                             IO_FILE_BUFFER_SIZE_DEFAULT),
                                        (short) 1,
                                        (short) 1,
                                        buffersize);
                                        buffersize);
     out.write(src, 0, len);
     out.write(src, 0, len);
@@ -392,6 +393,45 @@ public class ContractTestUtils extends Assert {
     rejectRootOperation(path, false);
     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) {
   public static void noteAction(String action) {
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {

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

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

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

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

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

@@ -50,7 +50,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.ConcurrentModificationException;
 import java.util.LinkedHashSet;
 import java.util.LinkedHashSet;
-import java.util.List;
 import java.util.Set;
 import java.util.Set;
 
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 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);
     ugi.addToken(new Text("regular-token"), token);
 
 
     // Now add cloned private 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
     // Ensure only non-private tokens are returned
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();
     assertEquals(1, tokens.size());
     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
    * This test checks a race condition between getting and adding tokens for
    * the current user.  Calling UserGroupInformation.getCurrentUser() returns
    * the current user.  Calling UserGroupInformation.getCurrentUser() returns

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSLinkResolver;
 import org.apache.hadoop.fs.FSLinkResolver;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemLinkResolver;
 import org.apache.hadoop.fs.FileSystemLinkResolver;
@@ -2204,6 +2205,35 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.listEncryptionZones();
     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
   @Override
   public void setXAttr(Path path, final String name, final byte[] value,
   public void setXAttr(Path path, final String name, final byte[] value,
       final EnumSet<XAttrSetFlag> flag) throws IOException {
       final EnumSet<XAttrSetFlag> flag) throws IOException {

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

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

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

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

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

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

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


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

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

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

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

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


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

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

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

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

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

@@ -1,175 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
-http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-alpha2-SNAPSHOT</version>
-    <relativePath>../../../../../hadoop-project</relativePath>
-  </parent>
-
-  <groupId>org.apache.hadoop.contrib</groupId>
-  <artifactId>hadoop-hdfs-bkjournal</artifactId>
-  <version>3.0.0-alpha2-SNAPSHOT</version>
-  <description>Apache Hadoop HDFS BookKeeper Journal</description>
-  <name>Apache Hadoop HDFS BookKeeper Journal</name>
-  <packaging>jar</packaging>
-
-  <properties>
-    <hadoop.component>hdfs</hadoop.component>
-    <hadoop.common.build.dir>${basedir}/../../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency> 
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency> 
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency> 
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.bookkeeper</groupId>
-      <artifactId>bookkeeper-server</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/../../../../../hadoop-common-project/hadoop-common/src/main/proto</param>
-                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto</param>
-                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs/src/main/proto</param>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>bkjournal.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <configuration>
-          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-          <excludes>
-            <exclude>dev-support/findbugsExcludeFile.xml</exclude>
-          </excludes>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-  <profiles>
-    <profile>
-      <id>dist</id>
-      <build>
-        <plugins>
-          <plugin>
-            <artifactId>maven-dependency-plugin</artifactId>
-            <version>2.8</version>
-            <executions>
-              <execution>
-                <id>dist</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>copy</goal>
-                </goals>
-                <configuration>
-                  <artifactItems>
-                    <artifactItem>
-                      <groupId>org.apache.bookkeeper</groupId>
-                      <artifactId>bookkeeper-server</artifactId>
-                      <type>jar</type>
-                    </artifactItem>
-                  </artifactItems>
-                  <outputDirectory>${project.build.directory}/lib</outputDirectory>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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