Browse Source

Merge branch 'trunk' into HADOOP-12756

Kai Zheng 8 years ago
parent
commit
dbb28eb1f2
100 changed files with 3793 additions and 2414 deletions
  1. 9 9
      .gitattributes
  2. 9 1
      hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
  3. 0 0
      hadoop-build-tools/src/main/resources/checkstyle/suppressions.xml
  4. 3 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
  5. 107 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  6. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
  7. 78 40
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java
  8. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  9. 7 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
  10. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  11. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  12. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
  13. 0 6
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  14. 1 0
      hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md
  15. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
  16. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
  17. 154 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestConnectionRetryPolicy.java
  18. 10 28
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
  19. 6 230
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
  20. 0 13
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
  21. 23 219
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
  22. 29 15
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java
  23. 166 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestReuseRpcConnections.java
  24. 59 18
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
  25. 48 26
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
  26. 103 188
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
  27. 23 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  28. 2 2
      hadoop-common-project/hadoop-common/src/test/proto/test.proto
  29. 7 1
      hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
  30. 32 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  31. 27 60
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  32. 48 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  33. 2 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  34. 62 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
  35. 20 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
  36. 17 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
  37. 2 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
  38. 59 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  40. 15 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  42. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
  43. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  44. 51 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
  45. 68 67
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  46. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java
  47. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  48. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  49. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
  50. 479 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
  51. 417 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
  52. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
  53. 252 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
  54. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaHandler.java
  55. 51 273
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
  56. 0 86
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java
  57. 169 201
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  58. 12 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
  59. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
  60. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  61. 49 25
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  62. 44 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
  63. 199 369
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  64. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  65. 145 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  66. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
  67. 17 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
  68. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  69. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  70. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  71. 10 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
  72. 22 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  73. 108 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
  74. 122 61
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
  75. 15 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
  76. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  77. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
  78. 0 119
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
  79. 26 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  80. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  81. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java
  82. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
  83. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
  84. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java
  85. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  86. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
  87. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
  88. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
  89. 24 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
  90. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java
  91. 21 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java
  92. 15 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
  93. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
  94. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
  95. 26 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
  96. 0 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
  97. 124 35
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
  98. 16 0
      hadoop-tools/hadoop-openstack/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
  99. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
  100. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java

+ 9 - 9
.gitattributes

@@ -1,15 +1,15 @@
 # Auto detect text files and perform LF normalization
 *        text=auto
 
-*.cs     text diff=csharp
-*.java   text diff=java
-*.html   text diff=html
-*.py     text diff=python
-*.pl     text diff=perl
-*.pm     text diff=perl
-*.css    text
-*.js     text
-*.sql    text
+*.cs     text diff=csharp eol=lf
+*.java   text diff=java eol=lf
+*.html   text diff=html eol=lf
+*.py     text diff=python eol=lf
+*.pl     text diff=perl eol=lf
+*.pm     text diff=perl eol=lf
+*.css    text eol=lf
+*.js     text eol=lf
+*.sql    text eol=lf
 
 *.sh     text eol=lf
 

+ 9 - 1
hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml

@@ -49,6 +49,10 @@
 
 <module name="Checker">
 
+    <module name="SuppressWarningsFilter"/>
+    <module name="SuppressionCommentFilter"/>
+    <module name="SuppressWithNearbyCommentFilter"/>
+
     <!-- Checks that a package.html file exists for each package.     -->
     <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
     <module name="JavadocPackage"/>
@@ -67,6 +71,10 @@
 
     <module name="TreeWalker">
 
+        <module name="SuppressWarningsHolder"/>
+        <module name="FileContentsHolder"/>
+
+
         <!-- Checks for Javadoc comments.                     -->
         <!-- See http://checkstyle.sf.net/config_javadoc.html -->
         <module name="JavadocType">
@@ -180,7 +188,7 @@
         <module name="Indentation">
             <property name="basicOffset" value="2" />
             <property name="caseIndent" value="0" />
-        </module> 
+        </module>
         <!--<module name="TodoComment"/>-->
         <module name="UpperEll"/>
 

+ 0 - 0
dev-support/checkstyle/suppressions.xml → hadoop-build-tools/src/main/resources/checkstyle/suppressions.xml


+ 3 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java

@@ -438,6 +438,9 @@ public class AuthenticationFilter implements Filter {
       for (Cookie cookie : cookies) {
         if (cookie.getName().equals(AuthenticatedURL.AUTH_COOKIE)) {
           tokenStr = cookie.getValue();
+          if (tokenStr.isEmpty()) {
+            throw new AuthenticationException("Unauthorized access");
+          }
           try {
             tokenStr = signer.verifyAndExtract(tokenStr);
           } catch (SignerException ex) {

+ 107 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -73,6 +73,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.core.Tracer;
 import org.apache.htrace.core.TraceScope;
 
+import com.google.common.base.Preconditions;
 import com.google.common.annotations.VisibleForTesting;
 
 import static com.google.common.base.Preconditions.checkArgument;
@@ -1530,7 +1531,68 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
   public abstract FileStatus[] listStatus(Path f) throws FileNotFoundException, 
                                                          IOException;
-    
+
+  /**
+   * Represents a batch of directory entries when iteratively listing a
+   * directory. This is a private API not meant for use by end users.
+   * <p>
+   * For internal use by FileSystem subclasses that override
+   * {@link FileSystem#listStatusBatch(Path, byte[])} to implement iterative
+   * listing.
+   */
+  @InterfaceAudience.Private
+  public static class DirectoryEntries {
+    private final FileStatus[] entries;
+    private final byte[] token;
+    private final boolean hasMore;
+
+    public DirectoryEntries(FileStatus[] entries, byte[] token, boolean
+        hasMore) {
+      this.entries = entries;
+      if (token != null) {
+        this.token = token.clone();
+      } else {
+        this.token = null;
+      }
+      this.hasMore = hasMore;
+    }
+
+    public FileStatus[] getEntries() {
+      return entries;
+    }
+
+    public byte[] getToken() {
+      return token;
+    }
+
+    public boolean hasMore() {
+      return hasMore;
+    }
+  }
+
+  /**
+   * Given an opaque iteration token, return the next batch of entries in a
+   * directory. This is a private API not meant for use by end users.
+   * <p>
+   * This method should be overridden by FileSystem subclasses that want to
+   * use the generic {@link FileSystem#listStatusIterator(Path)} implementation.
+   * @param f Path to list
+   * @param token opaque iteration token returned by previous call, or null
+   *              if this is the first call.
+   * @return
+   * @throws FileNotFoundException
+   * @throws IOException
+   */
+  @InterfaceAudience.Private
+  protected DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+      FileNotFoundException, IOException {
+    // The default implementation returns the entire listing as a single batch.
+    // Thus, there is never a second batch, and no need to respect the passed
+    // token or set a token in the returned DirectoryEntries.
+    FileStatus[] listing = listStatus(f);
+    return new DirectoryEntries(listing, null, false);
+  }
+
   /*
    * Filter files/directories in the given path using the user-supplied path
    * filter. Results are added to the given array <code>results</code>.
@@ -1766,6 +1828,49 @@ public abstract class FileSystem extends Configured implements Closeable {
     };
   }
 
+  /**
+   * Generic iterator for implementing {@link #listStatusIterator(Path)}.
+   */
+  private class DirListingIterator<T extends FileStatus> implements
+      RemoteIterator<T> {
+
+    private final Path path;
+    private DirectoryEntries entries;
+    private int i = 0;
+
+    DirListingIterator(Path path) {
+      this.path = path;
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      if (entries == null) {
+        fetchMore();
+      }
+      return i < entries.getEntries().length ||
+          entries.hasMore();
+    }
+
+    private void fetchMore() throws IOException {
+      byte[] token = null;
+      if (entries != null) {
+        token = entries.getToken();
+      }
+      entries = listStatusBatch(path, token);
+      i = 0;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public T next() throws IOException {
+      Preconditions.checkState(hasNext(), "No more items in iterator");
+      if (i == entries.getEntries().length) {
+        fetchMore();
+      }
+      return (T)entries.getEntries()[i++];
+    }
+  }
+
   /**
    * Returns a remote iterator so that followup calls are made on demand
    * while consuming the entries. Each file system implementation should
@@ -1779,23 +1884,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
   public RemoteIterator<FileStatus> listStatusIterator(final Path p)
   throws FileNotFoundException, IOException {
-    return new RemoteIterator<FileStatus>() {
-      private final FileStatus[] stats = listStatus(p);
-      private int i = 0;
-
-      @Override
-      public boolean hasNext() {
-        return i<stats.length;
-      }
-
-      @Override
-      public FileStatus next() throws IOException {
-        if (!hasNext()) {
-          throw new NoSuchElementException("No more entry in " + p);
-        }
-        return stats[i++];
-      }
-    };
+    return new DirListingIterator<>(p);
   }
 
   /**

+ 14 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java

@@ -183,6 +183,20 @@ public class RetryPolicies {
       return new RetryAction(RetryAction.RetryDecision.FAIL, 0, "try once " +
           "and fail.");
     }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else {
+        return obj != null && obj.getClass() == this.getClass();
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      return this.getClass().hashCode();
+    }
   }
 
   static class RetryForever implements RetryPolicy {

+ 78 - 40
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicies.MultipleLinearRandomRetry;
 import org.apache.hadoop.ipc.RemoteException;
 
 import com.google.protobuf.ServiceException;
@@ -79,48 +80,85 @@ public class RetryUtils {
       //no retry
       return RetryPolicies.TRY_ONCE_THEN_FAIL;
     } else {
-      return new RetryPolicy() {
-        @Override
-        public RetryAction shouldRetry(Exception e, int retries, int failovers,
-            boolean isMethodIdempotent) throws Exception {
-          if (e instanceof ServiceException) {
-            //unwrap ServiceException
-            final Throwable cause = e.getCause();
-            if (cause != null && cause instanceof Exception) {
-              e = (Exception)cause;
-            }
-          }
-
-          //see (1) and (2) in the javadoc of this method.
-          final RetryPolicy p;
-          if (e instanceof RetriableException
-              || RetryPolicies.getWrappedRetriableException(e) != null) {
-            // RetriableException or RetriableException wrapped
-            p = multipleLinearRandomRetry;
-          } else if (e instanceof RemoteException) {
-            final RemoteException re = (RemoteException)e;
-            p = remoteExceptionToRetry.equals(re.getClassName())?
-                multipleLinearRandomRetry: RetryPolicies.TRY_ONCE_THEN_FAIL;
-          } else if (e instanceof IOException || e instanceof ServiceException) {
-            p = multipleLinearRandomRetry;
-          } else { //non-IOException
-            p = RetryPolicies.TRY_ONCE_THEN_FAIL;
-          }
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("RETRY " + retries + ") policy="
-                + p.getClass().getSimpleName() + ", exception=" + e);
-          }
-          return p.shouldRetry(e, retries, failovers, isMethodIdempotent);
-        }
+      return new WrapperRetryPolicy(
+          (MultipleLinearRandomRetry) multipleLinearRandomRetry,
+          remoteExceptionToRetry);
+    }
+  }
+
+  private static final class WrapperRetryPolicy implements RetryPolicy {
+    private MultipleLinearRandomRetry multipleLinearRandomRetry;
+    private String remoteExceptionToRetry;
 
-        @Override
-        public String toString() {
-          return "RetryPolicy[" + multipleLinearRandomRetry + ", "
-              + RetryPolicies.TRY_ONCE_THEN_FAIL.getClass().getSimpleName()
-              + "]";
+    private WrapperRetryPolicy(
+        final MultipleLinearRandomRetry multipleLinearRandomRetry,
+        final String remoteExceptionToRetry) {
+      this.multipleLinearRandomRetry = multipleLinearRandomRetry;
+      this.remoteExceptionToRetry = remoteExceptionToRetry;
+    }
+
+    @Override
+    public RetryAction shouldRetry(Exception e, int retries, int failovers,
+        boolean isMethodIdempotent) throws Exception {
+      if (e instanceof ServiceException) {
+        //unwrap ServiceException
+        final Throwable cause = e.getCause();
+        if (cause != null && cause instanceof Exception) {
+          e = (Exception)cause;
         }
-      };
+      }
+
+      //see (1) and (2) in the javadoc of this method.
+      final RetryPolicy p;
+      if (e instanceof RetriableException
+          || RetryPolicies.getWrappedRetriableException(e) != null) {
+        // RetriableException or RetriableException wrapped
+        p = multipleLinearRandomRetry;
+      } else if (e instanceof RemoteException) {
+        final RemoteException re = (RemoteException)e;
+        p = re.getClassName().equals(remoteExceptionToRetry)
+            ? multipleLinearRandomRetry : RetryPolicies.TRY_ONCE_THEN_FAIL;
+      } else if (e instanceof IOException || e instanceof ServiceException) {
+        p = multipleLinearRandomRetry;
+      } else { //non-IOException
+        p = RetryPolicies.TRY_ONCE_THEN_FAIL;
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RETRY " + retries + ") policy="
+            + p.getClass().getSimpleName() + ", exception=" + e);
+      }
+      return p.shouldRetry(e, retries, failovers, isMethodIdempotent);
+    }
+
+    /**
+     * remoteExceptionToRetry is ignored as part of equals since it does not
+     * affect connection failure handling.
+     */
+    @Override
+    public boolean equals(final Object obj) {
+      if (obj == this) {
+        return true;
+      } else {
+        return (obj instanceof WrapperRetryPolicy)
+            && this.multipleLinearRandomRetry
+                .equals(((WrapperRetryPolicy) obj).multipleLinearRandomRetry);
+      }
+    }
+
+    /**
+     * Similarly, remoteExceptionToRetry is ignored as part of hashCode since it
+     * does not affect connection failure handling.
+     */
+    @Override
+    public int hashCode() {
+      return multipleLinearRandomRetry.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return "RetryPolicy[" + multipleLinearRandomRetry + ", "
+          + RetryPolicies.TRY_ONCE_THEN_FAIL.getClass().getSimpleName() + "]";
     }
   }
 

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -60,7 +60,7 @@ public class ProtobufRpcEngine implements RpcEngine {
   private static final ThreadLocal<AsyncGet<Message, Exception>>
       ASYNC_RETURN_MESSAGE = new ThreadLocal<>();
 
-  static { // Register the rpcRequest deserializer for WritableRpcEngine 
+  static { // Register the rpcRequest deserializer for ProtobufRpcEngine
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
         RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcProtobufRequest.class,
         new Server.ProtoBufRpcInvoker());
@@ -194,7 +194,8 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       
       if (args.length != 2) { // RpcController + Message
-        throw new ServiceException("Too many parameters for request. Method: ["
+        throw new ServiceException(
+            "Too many or few parameters for request. Method: ["
             + method.getName() + "]" + ", Expected: 2, Actual: "
             + args.length);
       }

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.ipc;
 
+import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
@@ -26,7 +28,6 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.NoRouteToHostException;
 import java.net.SocketTimeoutException;
-import java.io.*;
 import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -37,11 +38,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.net.SocketFactory;
 
-import org.apache.commons.logging.*;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
@@ -54,7 +56,6 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 
@@ -87,7 +88,7 @@ public class RPC {
     RPC_WRITABLE ((short) 2),        // Use WritableRpcEngine 
     RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
     final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
-    public final short value; //TODO make it private
+    private final short value;
 
     RpcKind(short val) {
       this.value = val;

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

@@ -237,14 +237,14 @@ public abstract class Server {
   static class RpcKindMapValue {
     final Class<? extends Writable> rpcRequestWrapperClass;
     final RpcInvoker rpcInvoker;
+
     RpcKindMapValue (Class<? extends Writable> rpcRequestWrapperClass,
           RpcInvoker rpcInvoker) {
       this.rpcInvoker = rpcInvoker;
       this.rpcRequestWrapperClass = rpcRequestWrapperClass;
     }   
   }
-  static Map<RPC.RpcKind, RpcKindMapValue> rpcKindMap = new
-      HashMap<RPC.RpcKind, RpcKindMapValue>(4);
+  static Map<RPC.RpcKind, RpcKindMapValue> rpcKindMap = new HashMap<>(4);
   
   
 

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

@@ -730,7 +730,7 @@ public class UserGroupInformation {
    * 
    * @param user                The principal name to load from the ticket
    *                            cache
-   * @param ticketCachePath     the path to the ticket cache file
+   * @param ticketCache     the path to the ticket cache file
    *
    * @throws IOException        if the kerberos login fails
    */
@@ -790,7 +790,7 @@ public class UserGroupInformation {
   /**
    * Create a UserGroupInformation from a Subject with Kerberos principal.
    *
-   * @param user                The KerberosPrincipal to use in UGI
+   * @param subject             The KerberosPrincipal to use in UGI
    *
    * @throws IOException        if the kerberos login fails
    */

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

@@ -226,7 +226,7 @@ public class RunJar {
 
     unJar(file, workDir);
 
-    ClassLoader loader = createClassLoader(workDir);
+    ClassLoader loader = createClassLoader(file, workDir);
 
     Thread.currentThread().setContextClassLoader(loader);
     Class<?> mainClass = Class.forName(mainClassName, true, loader);
@@ -250,13 +250,14 @@ public class RunJar {
    * the user jar as well as the HADOOP_CLASSPATH. Otherwise, it creates a
    * classloader that simply adds the user jar to the classpath.
    */
-  private ClassLoader createClassLoader(final File workDir)
+  private ClassLoader createClassLoader(File file, final File workDir)
       throws MalformedURLException {
     ClassLoader loader;
     // see if the client classloader is enabled
     if (useClientClassLoader()) {
       StringBuilder sb = new StringBuilder();
       sb.append(workDir).append("/").
+          append(File.pathSeparator).append(file).
           append(File.pathSeparator).append(workDir).append("/classes/").
           append(File.pathSeparator).append(workDir).append("/lib/*");
       // HADOOP_CLASSPATH is added to the client classpath
@@ -276,6 +277,7 @@ public class RunJar {
     } else {
       List<URL> classPath = new ArrayList<>();
       classPath.add(new File(workDir + "/").toURI().toURL());
+      classPath.add(file.toURI().toURL());
       classPath.add(new File(workDir, "classes/").toURI().toURL());
       File[] libs = new File(workDir, "lib").listFiles();
       if (libs != null) {

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

@@ -798,12 +798,6 @@
   </description>
 </property>
 
-<property>
-  <name>fs.swift.impl</name>
-  <value>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</value>
-  <description>The implementation class of the OpenStack Swift Filesystem</description>
-</property>
-
 <property>
   <name>fs.automatic.close</name>
   <value>true</value>

+ 1 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md

@@ -91,6 +91,7 @@ When running benchmarks with the above operation(s), please provide operation-sp
 The benchmark measures the number of operations performed by the name-node per second. Specifically, for each operation tested, it reports the total running time in seconds (_Elapsed Time_), operation throughput (_Ops per sec_), and average time for the operations (_Average Time_). The higher, the better.
 
 Following is a sample reports by running following commands that opens 100K files with 1K threads against a remote name-node. See [HDFS scalability: the limits to growth](https://www.usenix.org/legacy/publications/login/2010-04/openpdfs/shvachko.pdf) for real-world benchmark stats.
+
 ```
 $ hadoop org.apache.hadoop.hdfs.server.namenode.NNThroughputBenchmark -fs hdfs://nameservice:9000 -op open -threads 1000 -files 100000
 

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java

@@ -103,6 +103,7 @@ public class TestFilterFileSystem {
     public void processDeleteOnExit();
     public FsStatus getStatus();
     public FileStatus[] listStatus(Path f, PathFilter filter);
+    public FileStatus[] listStatusBatch(Path f, byte[] token);
     public FileStatus[] listStatus(Path[] files);
     public FileStatus[] listStatus(Path[] files, PathFilter filter);
     public FileStatus[] globStatus(Path pathPattern);

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java

@@ -115,6 +115,7 @@ public class TestHarFileSystem {
     public QuotaUsage getQuotaUsage(Path f);
     public FsStatus getStatus();
     public FileStatus[] listStatus(Path f, PathFilter filter);
+    public FileStatus[] listStatusBatch(Path f, byte[] token);
     public FileStatus[] listStatus(Path[] files);
     public FileStatus[] listStatus(Path[] files, PathFilter filter);
     public FileStatus[] globStatus(Path pathPattern);

+ 154 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestConnectionRetryPolicy.java

@@ -0,0 +1,154 @@
+/**
+ * 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.io.retry;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RetriableException;
+import org.apache.hadoop.ipc.RpcNoSuchMethodException;
+import org.junit.Test;
+
+/**
+ * This class mainly tests behaviors of various retry policies in connection
+ * level.
+ */
+public class TestConnectionRetryPolicy {
+  private static RetryPolicy getDefaultRetryPolicy(
+      final boolean defaultRetryPolicyEnabled,
+      final String defaultRetryPolicySpec,
+      final String remoteExceptionToRetry) {
+    return getDefaultRetryPolicy(
+        new Configuration(),
+        defaultRetryPolicyEnabled,
+        defaultRetryPolicySpec,
+        remoteExceptionToRetry);
+  }
+
+  private static RetryPolicy getDefaultRetryPolicy(
+      final boolean defaultRetryPolicyEnabled,
+      final String defaultRetryPolicySpec) {
+    return getDefaultRetryPolicy(
+        new Configuration(),
+        defaultRetryPolicyEnabled,
+        defaultRetryPolicySpec,
+        "");
+  }
+
+  public static RetryPolicy getDefaultRetryPolicy(
+      final Configuration conf,
+      final boolean defaultRetryPolicyEnabled,
+      final String defaultRetryPolicySpec,
+      final String remoteExceptionToRetry) {
+    return RetryUtils.getDefaultRetryPolicy(
+        conf,
+        "org.apache.hadoop.io.retry.TestConnectionRetryPolicy.No.Such.Key",
+        defaultRetryPolicyEnabled,
+        "org.apache.hadoop.io.retry.TestConnectionRetryPolicy.No.Such.Key",
+        defaultRetryPolicySpec,
+        "");
+  }
+
+  @Test(timeout = 60000)
+  public void testDefaultRetryPolicyEquivalence() {
+    RetryPolicy rp1 = null;
+    RetryPolicy rp2 = null;
+    RetryPolicy rp3 = null;
+
+    /* test the same setting */
+    rp1 = getDefaultRetryPolicy(true, "10000,2");
+    rp2 = getDefaultRetryPolicy(true, "10000,2");
+    rp3 = getDefaultRetryPolicy(true, "10000,2");
+    verifyRetryPolicyEquivalence(new RetryPolicy[] {rp1, rp2, rp3});
+
+    /* test different remoteExceptionToRetry */
+    rp1 = getDefaultRetryPolicy(
+        true,
+        "10000,2",
+        new RemoteException(
+            PathIOException.class.getName(),
+            "path IO exception").getClassName());
+    rp2 = getDefaultRetryPolicy(
+        true,
+        "10000,2",
+        new RemoteException(
+            RpcNoSuchMethodException.class.getName(),
+            "no such method exception").getClassName());
+    rp3 = getDefaultRetryPolicy(
+        true,
+        "10000,2",
+        new RemoteException(
+            RetriableException.class.getName(),
+            "retriable exception").getClassName());
+    verifyRetryPolicyEquivalence(new RetryPolicy[] {rp1, rp2, rp3});
+
+    /* test enabled and different specifications */
+    rp1 = getDefaultRetryPolicy(true, "20000,3");
+    rp2 = getDefaultRetryPolicy(true, "30000,4");
+    assertNotEquals("should not be equal", rp1, rp2);
+    assertNotEquals(
+        "should not have the same hash code",
+        rp1.hashCode(),
+        rp2.hashCode());
+
+    /* test disabled and the same specifications */
+    rp1 = getDefaultRetryPolicy(false, "40000,5");
+    rp2 = getDefaultRetryPolicy(false, "40000,5");
+    assertEquals("should be equal", rp1, rp2);
+    assertEquals(
+        "should have the same hash code",
+        rp1, rp2);
+
+    /* test the disabled and different specifications */
+    rp1 = getDefaultRetryPolicy(false, "50000,6");
+    rp2 = getDefaultRetryPolicy(false, "60000,7");
+    assertEquals("should be equal", rp1, rp2);
+    assertEquals(
+        "should have the same hash code",
+        rp1, rp2);
+  }
+
+  public static RetryPolicy newTryOnceThenFail() {
+    return new RetryPolicies.TryOnceThenFail();
+  }
+
+  @Test(timeout = 60000)
+  public void testTryOnceThenFailEquivalence() throws Exception {
+    final RetryPolicy rp1 = newTryOnceThenFail();
+    final RetryPolicy rp2 = newTryOnceThenFail();
+    final RetryPolicy rp3 = newTryOnceThenFail();
+    verifyRetryPolicyEquivalence(new RetryPolicy[] {rp1, rp2, rp3});
+  }
+
+  private void verifyRetryPolicyEquivalence(RetryPolicy[] polices) {
+    for (int i = 0; i < polices.length; i++) {
+      for (int j = 0; j < polices.length; j++) {
+        if (i != j) {
+          assertEquals("should be equal", polices[i], polices[j]);
+          assertEquals(
+              "should have the same hash code",
+              polices[i].hashCode(),
+              polices[j].hashCode());
+        }
+      }
+    }
+  }
+}

+ 10 - 28
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java

@@ -17,13 +17,8 @@
  */
 package org.apache.hadoop.ipc;
 
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadMXBean;
-import java.net.InetSocketAddress;
-import java.security.PrivilegedExceptionAction;
-import java.util.concurrent.atomic.AtomicLong;
-
+import com.google.common.base.Joiner;
+import com.google.protobuf.BlockingService;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -34,7 +29,6 @@ import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ipc.RPC.Server;
-import org.apache.hadoop.ipc.TestRPC.TestProtocol;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
 import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
@@ -45,8 +39,12 @@ import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import com.google.common.base.Joiner;
-import com.google.protobuf.BlockingService;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * Benchmark for protobuf RPC.
@@ -68,7 +66,7 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
     public int secondsToRun = 15;
     private int msgSize = 1024;
     public Class<? extends RpcEngine> rpcEngine =
-      WritableRpcEngine.class;
+        ProtobufRpcEngine.class;
     
     private MyOptions(String args[]) {
       try {
@@ -135,7 +133,7 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
       
       opts.addOption(
           OptionBuilder.withLongOpt("engine").hasArg(true)
-          .withArgName("writable|protobuf")
+          .withArgName("protobuf")
           .withDescription("engine to use")
           .create('e'));
       
@@ -184,8 +182,6 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
         String eng = line.getOptionValue('e');
         if ("protobuf".equals(eng)) {
           rpcEngine = ProtobufRpcEngine.class;
-        } else if ("writable".equals(eng)) {
-          rpcEngine = WritableRpcEngine.class;
         } else {
           throw new ParseException("invalid engine: " + eng);
         }
@@ -237,11 +233,6 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
       server = new RPC.Builder(conf).setProtocol(TestRpcService.class)
           .setInstance(service).setBindAddress(opts.host).setPort(opts.getPort())
           .setNumHandlers(opts.serverThreads).setVerbose(false).build();
-    } else if (opts.rpcEngine == WritableRpcEngine.class) {
-      server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-          .setInstance(new TestRPC.TestImpl()).setBindAddress(opts.host)
-          .setPort(opts.getPort()).setNumHandlers(opts.serverThreads)
-          .setVerbose(false).build();
     } else {
       throw new RuntimeException("Bad engine: " + opts.rpcEngine);
     }
@@ -399,15 +390,6 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
           return responseProto.getMessage();
         }
       };
-    } else if (opts.rpcEngine == WritableRpcEngine.class) {
-      final TestProtocol proxy = RPC.getProxy(
-          TestProtocol.class, TestProtocol.versionID, addr, conf);
-      return new RpcServiceWrapper() {
-        @Override
-        public String doEcho(String msg) throws Exception {
-          return proxy.echo(msg);
-        }
-      };
     } else {
       throw new RuntimeException("unsupported engine: " + opts.rpcEngine);
     }

+ 6 - 230
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java

@@ -17,252 +17,28 @@
  */
 package org.apache.hadoop.ipc;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
-import org.apache.hadoop.net.NetUtils;
-import org.junit.Before;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
-import com.google.protobuf.BlockingService;
 
 public class TestMultipleProtocolServer extends TestRpcBase {
-  private static InetSocketAddress addr;
-  private static RPC.Server server;
-
-  private static Configuration conf = new Configuration();
-  
-  
-  @ProtocolInfo(protocolName="Foo")
-  interface Foo0 extends VersionedProtocol {
-    public static final long versionID = 0L;
-    String ping() throws IOException;
-    
-  }
-  
-  @ProtocolInfo(protocolName="Foo")
-  interface Foo1 extends VersionedProtocol {
-    public static final long versionID = 1L;
-    String ping() throws IOException;
-    String ping2() throws IOException;
-  }
-  
-  @ProtocolInfo(protocolName="Foo")
-  interface FooUnimplemented extends VersionedProtocol {
-    public static final long versionID = 2L;
-    String ping() throws IOException;  
-  }
-  
-  interface Mixin extends VersionedProtocol{
-    public static final long versionID = 0L;
-    void hello() throws IOException;
-  }
-
-  interface Bar extends Mixin {
-    public static final long versionID = 0L;
-    int echo(int i) throws IOException;
-  }
-  
-  class Foo0Impl implements Foo0 {
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return Foo0.versionID;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      Class<? extends VersionedProtocol> inter;
-      try {
-        inter = (Class<? extends VersionedProtocol>)getClass().
-                                          getGenericInterfaces()[0];
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-          getProtocolVersion(protocol, clientVersion), inter);
-    }
-
-    @Override
-    public String ping() {
-      return "Foo0";     
-    }
-    
-  }
-  
-  class Foo1Impl implements Foo1 {
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return Foo1.versionID;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      Class<? extends VersionedProtocol> inter;
-      try {
-        inter = (Class<? extends VersionedProtocol>)getClass().
-                                        getGenericInterfaces()[0];
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-          getProtocolVersion(protocol, clientVersion), inter);
-    }
-
-    @Override
-    public String ping() {
-      return "Foo1";
-    }
 
-    @Override
-    public String ping2() {
-      return "Foo1";
-      
-    }
-    
-  }
-
-  
-  class BarImpl implements Bar {
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return Bar.versionID;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      Class<? extends VersionedProtocol> inter;
-      try {
-        inter = (Class<? extends VersionedProtocol>)getClass().
-                                          getGenericInterfaces()[0];
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-          getProtocolVersion(protocol, clientVersion), inter);
-    }
-
-    @Override
-    public int echo(int i) {
-      return i;
-    }
-
-    @Override
-    public void hello() {
+  private static RPC.Server server;
 
-      
-    }
-  }
   @Before
   public void setUp() throws Exception {
-    // create a server with two handlers
-    server = new RPC.Builder(conf).setProtocol(Foo0.class)
-        .setInstance(new Foo0Impl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
-    
-    
-    // Add Protobuf server
-    // Create server side implementation
-    PBServerImpl pbServerImpl = new PBServerImpl();
-    BlockingService service = TestProtobufRpcProto
-        .newReflectiveBlockingService(pbServerImpl);
-    server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
-        service);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
+    super.setupConf();
+
+    server = setupTestServer(conf, 2);
   }
-  
+
   @After
   public void tearDown() throws Exception {
     server.stop();
   }
 
-  @Test
-  public void test1() throws IOException {
-    ProtocolProxy<?> proxy;
-    proxy = RPC.getProtocolProxy(Foo0.class, Foo0.versionID, addr, conf);
 
-    Foo0 foo0 = (Foo0)proxy.getProxy(); 
-    Assert.assertEquals("Foo0", foo0.ping());
-    
-    
-    proxy = RPC.getProtocolProxy(Foo1.class, Foo1.versionID, addr, conf);
-    
-    
-    Foo1 foo1 = (Foo1)proxy.getProxy(); 
-    Assert.assertEquals("Foo1", foo1.ping());
-    Assert.assertEquals("Foo1", foo1.ping());
-    
-    
-    proxy = RPC.getProtocolProxy(Bar.class, Foo1.versionID, addr, conf);
-    
-    
-    Bar bar = (Bar)proxy.getProxy(); 
-    Assert.assertEquals(99, bar.echo(99));
-    
-    // Now test Mixin class method
-    
-    Mixin mixin = bar;
-    mixin.hello();
-  }
-  
-  
-  // Server does not implement the FooUnimplemented version of protocol Foo.
-  // See that calls to it fail.
-  @Test(expected=IOException.class)
-  public void testNonExistingProtocol() throws IOException {
-    ProtocolProxy<?> proxy;
-    proxy = RPC.getProtocolProxy(FooUnimplemented.class, 
-        FooUnimplemented.versionID, addr, conf);
-
-    FooUnimplemented foo = (FooUnimplemented)proxy.getProxy(); 
-    foo.ping();
-  }
-
-  /**
-   * getProtocolVersion of an unimplemented version should return highest version
-   * Similarly getProtocolSignature should work.
-   * @throws IOException
-   */
-  @Test
-  public void testNonExistingProtocol2() throws IOException {
-    ProtocolProxy<?> proxy;
-    proxy = RPC.getProtocolProxy(FooUnimplemented.class, 
-        FooUnimplemented.versionID, addr, conf);
-
-    FooUnimplemented foo = (FooUnimplemented)proxy.getProxy(); 
-    Assert.assertEquals(Foo1.versionID, 
-        foo.getProtocolVersion(RPC.getProtocolName(FooUnimplemented.class), 
-        FooUnimplemented.versionID));
-    foo.getProtocolSignature(RPC.getProtocolName(FooUnimplemented.class), 
-        FooUnimplemented.versionID, 0);
-  }
-  
-  @Test(expected=IOException.class)
-  public void testIncorrectServerCreation() throws IOException {
-    new RPC.Builder(conf).setProtocol(Foo1.class).setInstance(new Foo0Impl())
-        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(2).setVerbose(false)
-        .build();
-  } 
-  
   // Now test a PB service - a server  hosts both PB and Writable Rpcs.
   @Test
   public void testPBService() throws Exception {

+ 0 - 13
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java

@@ -25,19 +25,6 @@ import org.junit.Test;
 
 public class TestRPCCallBenchmark {
 
-  @Test(timeout=20000)
-  public void testBenchmarkWithWritable() throws Exception {
-    int rc = ToolRunner.run(new RPCCallBenchmark(),
-        new String[] {
-      "--clientThreads", "30",
-      "--serverThreads", "30",
-      "--time", "5",
-      "--serverReaderThreads", "4",
-      "--messageSize", "1024",
-      "--engine", "writable"});
-    assertEquals(0, rc);
-  }
-  
   @Test(timeout=20000)
   public void testBenchmarkWithProto() throws Exception {
     int rc = ToolRunner.run(new RPCCallBenchmark(),

+ 23 - 219
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java

@@ -18,28 +18,20 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-
-import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
-import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
-import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
-import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
-import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
 /** Unit test for supporting method-name based compatible RPCs. */
 public class TestRPCCompatibility {
   private static final String ADDRESS = "0.0.0.0";
@@ -49,7 +41,7 @@ public class TestRPCCompatibility {
 
   public static final Log LOG =
     LogFactory.getLog(TestRPCCompatibility.class);
-  
+
   private static Configuration conf = new Configuration();
 
   public interface TestProtocol0 extends VersionedProtocol {
@@ -120,6 +112,21 @@ public class TestRPCCompatibility {
   @Before
   public void setUp() {
     ProtocolSignature.resetCache();
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol0.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol1.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol2.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol3.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol4.class, ProtobufRpcEngine.class);
   }
   
   @After
@@ -133,117 +140,7 @@ public class TestRPCCompatibility {
       server = null;
     }
   }
-  
-  @Test  // old client vs new server
-  public void testVersion0ClientVersion1Server() throws Exception {
-    // create a server with two handlers
-    TestImpl1 impl = new TestImpl1();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol1.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    proxy = RPC.getProtocolProxy(
-        TestProtocol0.class, TestProtocol0.versionID, addr, conf);
-
-    TestProtocol0 proxy0 = (TestProtocol0)proxy.getProxy();
-    proxy0.ping();
-  }
-  
-  @Test  // old client vs new server
-  public void testVersion1ClientVersion0Server() throws Exception {
-    // create a server with two handlers
-    server = new RPC.Builder(conf).setProtocol(TestProtocol0.class)
-        .setInstance(new TestImpl0()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    proxy = RPC.getProtocolProxy(
-        TestProtocol1.class, TestProtocol1.versionID, addr, conf);
-
-    TestProtocol1 proxy1 = (TestProtocol1)proxy.getProxy();
-    proxy1.ping();
-    try {
-      proxy1.echo("hello");
-      fail("Echo should fail");
-    } catch(IOException e) {
-    }
-  }
-  
-  private class Version2Client {
 
-    private TestProtocol2 proxy2;
-    private ProtocolProxy<TestProtocol2> serverInfo;
-    
-    private Version2Client() throws IOException {
-      serverInfo =  RPC.getProtocolProxy(
-          TestProtocol2.class, TestProtocol2.versionID, addr, conf);
-      proxy2 = serverInfo.getProxy();
-    }
-    
-    public int echo(int value) throws IOException, NumberFormatException {
-      if (serverInfo.isMethodSupported("echo", int.class)) {
-System.out.println("echo int is supported");
-        return -value;  // use version 3 echo long
-      } else { // server is version 2
-System.out.println("echo int is NOT supported");
-        return Integer.parseInt(proxy2.echo(String.valueOf(value)));
-      }
-    }
-
-    public String echo(String value) throws IOException {
-      return proxy2.echo(value);
-    }
-
-    public void ping() throws IOException {
-      proxy2.ping();
-    }
-  }
-
-  @Test // Compatible new client & old server
-  public void testVersion2ClientVersion1Server() throws Exception {
-    // create a server with two handlers
-    TestImpl1 impl = new TestImpl1();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol1.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-
-    Version2Client client = new Version2Client();
-    client.ping();
-    assertEquals("hello", client.echo("hello"));
-    
-    // echo(int) is not supported by server, so returning 3
-    // This verifies that echo(int) and echo(String)'s hash codes are different
-    assertEquals(3, client.echo(3));
-  }
-  
-  @Test // equal version client and server
-  public void testVersion2ClientVersion2Server() throws Exception {
-    // create a server with two handlers
-    TestImpl2 impl = new TestImpl2();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    Version2Client client = new Version2Client();
-
-    client.ping();
-    assertEquals("hello", client.echo("hello"));
-    
-    // now that echo(int) is supported by the server, echo(int) should return -3
-    assertEquals(-3, client.echo(3));
-  }
-  
   public interface TestProtocol3 {
     int echo(String value);
     int echo(int value);
@@ -297,97 +194,4 @@ System.out.println("echo int is NOT supported");
     @Override
     int echo(int value)  throws IOException;
   }
-  
-  @Test
-  public void testVersionMismatch() throws IOException {
-    server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
-        .setInstance(new TestImpl2()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    TestProtocol4 proxy = RPC.getProxy(TestProtocol4.class,
-        TestProtocol4.versionID, addr, conf);
-    try {
-      proxy.echo(21);
-      fail("The call must throw VersionMismatch exception");
-    } catch (RemoteException ex) {
-      Assert.assertEquals(RPC.VersionMismatch.class.getName(), 
-          ex.getClassName());
-      Assert.assertTrue(ex.getErrorCode().equals(
-          RpcErrorCodeProto.ERROR_RPC_VERSION_MISMATCH));
-    }  catch (IOException ex) {
-      fail("Expected version mismatch but got " + ex);
-    }
-  }
-  
-  @Test
-  public void testIsMethodSupported() throws IOException {
-    server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
-        .setInstance(new TestImpl2()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    TestProtocol2 proxy = RPC.getProxy(TestProtocol2.class,
-        TestProtocol2.versionID, addr, conf);
-    boolean supported = RpcClientUtil.isMethodSupported(proxy,
-        TestProtocol2.class, RPC.RpcKind.RPC_WRITABLE,
-        RPC.getProtocolVersion(TestProtocol2.class), "echo");
-    Assert.assertTrue(supported);
-    supported = RpcClientUtil.isMethodSupported(proxy,
-        TestProtocol2.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-        RPC.getProtocolVersion(TestProtocol2.class), "echo");
-    Assert.assertFalse(supported);
-  }
-
-  /**
-   * Verify that ProtocolMetaInfoServerSideTranslatorPB correctly looks up
-   * the server registry to extract protocol signatures and versions.
-   */
-  @Test
-  public void testProtocolMetaInfoSSTranslatorPB() throws Exception {
-    TestImpl1 impl = new TestImpl1();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol1.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-
-    ProtocolMetaInfoServerSideTranslatorPB xlator = 
-        new ProtocolMetaInfoServerSideTranslatorPB(server);
-
-    GetProtocolSignatureResponseProto resp = xlator.getProtocolSignature(
-        null,
-        createGetProtocolSigRequestProto(TestProtocol1.class,
-            RPC.RpcKind.RPC_PROTOCOL_BUFFER));
-    //No signatures should be found
-    Assert.assertEquals(0, resp.getProtocolSignatureCount());
-    resp = xlator.getProtocolSignature(
-        null,
-        createGetProtocolSigRequestProto(TestProtocol1.class,
-            RPC.RpcKind.RPC_WRITABLE));
-    Assert.assertEquals(1, resp.getProtocolSignatureCount());
-    ProtocolSignatureProto sig = resp.getProtocolSignatureList().get(0);
-    Assert.assertEquals(TestProtocol1.versionID, sig.getVersion());
-    boolean found = false;
-    int expected = ProtocolSignature.getFingerprint(TestProtocol1.class
-        .getMethod("echo", String.class));
-    for (int m : sig.getMethodsList()) {
-      if (expected == m) {
-        found = true;
-        break;
-      }
-    }
-    Assert.assertTrue(found);
-  }
-  
-  private GetProtocolSignatureRequestProto createGetProtocolSigRequestProto(
-      Class<?> protocol, RPC.RpcKind rpcKind) {
-    GetProtocolSignatureRequestProto.Builder builder = 
-        GetProtocolSignatureRequestProto.newBuilder();
-    builder.setProtocol(protocol.getName());
-    builder.setRpcKind(rpcKind.toString());
-    return builder.build();
-  }
 }

+ 29 - 15
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java

@@ -18,9 +18,8 @@
 package org.apache.hadoop.ipc;
 
 import org.apache.hadoop.conf.Configuration;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
-import org.apache.hadoop.ipc.TestRPC.TestProtocol;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,30 +29,39 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.nio.channels.ClosedByInterruptException;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY;
+
 /**
  * tests that the proxy can be interrupted
  */
-public class TestRPCWaitForProxy extends Assert {
-  private static final String ADDRESS = "0.0.0.0";
+public class TestRPCWaitForProxy extends TestRpcBase {
   private static final Logger
       LOG = LoggerFactory.getLogger(TestRPCWaitForProxy.class);
 
   private static final Configuration conf = new Configuration();
 
+  @Before
+  public void setupProtocolEngine() {
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+  }
+
   /**
    * This tests that the time-bounded wait for a proxy operation works, and
    * times out.
    *
    * @throws Throwable any exception other than that which was expected
    */
-  @Test(timeout = 10000)
+  @Test(timeout = 50000)
   public void testWaitForProxy() throws Throwable {
     RpcThread worker = new RpcThread(0);
     worker.start();
     worker.join();
     Throwable caught = worker.getCaught();
-    assertNotNull("No exception was raised", caught);
-    if (!(caught instanceof ConnectException)) {
+    Throwable cause = caught.getCause();
+    Assert.assertNotNull("No exception was raised", cause);
+    if (!(cause instanceof ConnectException)) {
       throw caught;
     }
   }
@@ -69,11 +77,11 @@ public class TestRPCWaitForProxy extends Assert {
     RpcThread worker = new RpcThread(100);
     worker.start();
     Thread.sleep(1000);
-    assertTrue("worker hasn't started", worker.waitStarted);
+    Assert.assertTrue("worker hasn't started", worker.waitStarted);
     worker.interrupt();
     worker.join();
     Throwable caught = worker.getCaught();
-    assertNotNull("No exception was raised", caught);
+    Assert.assertNotNull("No exception was raised", caught);
     // looking for the root cause here, which can be wrapped
     // as part of the NetUtils work. Having this test look
     // a the type of exception there would be brittle to improvements
@@ -82,6 +90,8 @@ public class TestRPCWaitForProxy extends Assert {
     if (cause == null) {
       // no inner cause, use outer exception as root cause.
       cause = caught;
+    } else if (cause.getCause() != null) {
+      cause = cause.getCause();
     }
     if (!(cause instanceof InterruptedIOException)
         && !(cause instanceof ClosedByInterruptException)) {
@@ -112,12 +122,16 @@ public class TestRPCWaitForProxy extends Assert {
             IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
             connectRetries);
         waitStarted = true;
-        TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
-            TestProtocol.versionID,
-            new InetSocketAddress(ADDRESS, 20),
-            config,
-            15000L);
-        proxy.echo("");
+
+        short invalidPort = 20;
+        InetSocketAddress invalidAddress = new InetSocketAddress(ADDRESS,
+            invalidPort);
+        TestRpcBase.TestRpcService proxy = RPC.getProxy(
+            TestRpcBase.TestRpcService.class,
+            1L, invalidAddress, conf);
+        // Test echo method
+        proxy.echo(null, newEchoRequest("hello"));
+
       } catch (Throwable throwable) {
         caught = throwable;
       }

+ 166 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestReuseRpcConnections.java

@@ -0,0 +1,166 @@
+/**
+ * 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 static org.junit.Assert.*;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.TestConnectionRetryPolicy;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.TestRpcBase.TestRpcService;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * This class mainly tests behaviors of reusing RPC connections for various
+ * retry policies.
+ */
+public class TestReuseRpcConnections extends TestRpcBase {
+  @Before
+  public void setup() {
+    setupConf();
+  }
+
+  private static RetryPolicy getDefaultRetryPolicy(
+      final boolean defaultRetryPolicyEnabled,
+      final String defaultRetryPolicySpec) {
+    return TestConnectionRetryPolicy.getDefaultRetryPolicy(
+        conf,
+        defaultRetryPolicyEnabled,
+        defaultRetryPolicySpec,
+        "");
+  }
+
+  private static RetryPolicy getDefaultRetryPolicy(
+      final boolean defaultRetryPolicyEnabled,
+      final String defaultRetryPolicySpec,
+      final String remoteExceptionToRetry) {
+    return TestConnectionRetryPolicy.getDefaultRetryPolicy(
+        conf,
+        defaultRetryPolicyEnabled,
+        defaultRetryPolicySpec,
+        remoteExceptionToRetry);
+  }
+
+  @Test(timeout = 60000)
+  public void testDefaultRetryPolicyReuseConnections() throws Exception {
+    RetryPolicy rp1 = null;
+    RetryPolicy rp2 = null;
+    RetryPolicy rp3 = null;
+
+    /* test the same setting */
+    rp1 = getDefaultRetryPolicy(true, "10000,2");
+    rp2 = getDefaultRetryPolicy(true, "10000,2");
+    verifyRetryPolicyReuseConnections(rp1, rp2, RetryPolicies.RETRY_FOREVER);
+
+    /* test enabled and different specifications */
+    rp1 = getDefaultRetryPolicy(true, "20000,3");
+    rp2 = getDefaultRetryPolicy(true, "20000,3");
+    rp3 = getDefaultRetryPolicy(true, "30000,4");
+    verifyRetryPolicyReuseConnections(rp1, rp2, rp3);
+
+    /* test disabled and the same specifications */
+    rp1 = getDefaultRetryPolicy(false, "40000,5");
+    rp2 = getDefaultRetryPolicy(false, "40000,5");
+    verifyRetryPolicyReuseConnections(rp1, rp2, RetryPolicies.RETRY_FOREVER);
+
+    /* test disabled and different specifications */
+    rp1 = getDefaultRetryPolicy(false, "50000,6");
+    rp2 = getDefaultRetryPolicy(false, "60000,7");
+    verifyRetryPolicyReuseConnections(rp1, rp2, RetryPolicies.RETRY_FOREVER);
+
+    /* test different remoteExceptionToRetry */
+    rp1 = getDefaultRetryPolicy(
+        true,
+        "70000,8",
+        new RemoteException(
+            RpcNoSuchMethodException.class.getName(),
+            "no such method exception").getClassName());
+    rp2 = getDefaultRetryPolicy(
+        true,
+        "70000,8",
+        new RemoteException(
+            PathIOException.class.getName(),
+            "path IO exception").getClassName());
+    verifyRetryPolicyReuseConnections(rp1, rp2, RetryPolicies.RETRY_FOREVER);
+  }
+
+  @Test(timeout = 60000)
+  public void testRetryPolicyTryOnceThenFail() throws Exception {
+    final RetryPolicy rp1 = TestConnectionRetryPolicy.newTryOnceThenFail();
+    final RetryPolicy rp2 = TestConnectionRetryPolicy.newTryOnceThenFail();
+    verifyRetryPolicyReuseConnections(rp1, rp2, RetryPolicies.RETRY_FOREVER);
+  }
+
+  private void verifyRetryPolicyReuseConnections(
+      final RetryPolicy retryPolicy1,
+      final RetryPolicy retryPolicy2,
+      final RetryPolicy anotherRetryPolicy) throws Exception {
+    final Server server = setupTestServer(conf, 2);
+    final Configuration newConf = new Configuration(conf);
+    newConf.set(
+        CommonConfigurationKeysPublic
+          .HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY,
+        "");
+    Client client = null;
+    TestRpcService proxy1 = null;
+    TestRpcService proxy2 = null;
+    TestRpcService proxy3 = null;
+
+    try {
+      proxy1 = getClient(addr, newConf, retryPolicy1);
+      proxy1.ping(null, newEmptyRequest());
+      client = ProtobufRpcEngine.getClient(newConf);
+      final Set<ConnectionId> conns = client.getConnectionIds();
+      assertEquals("number of connections in cache is wrong", 1, conns.size());
+
+      /*
+       * another equivalent retry policy, reuse connection
+       */
+      proxy2 = getClient(addr, newConf, retryPolicy2);
+      proxy2.ping(null, newEmptyRequest());
+      assertEquals("number of connections in cache is wrong", 1, conns.size());
+
+      /*
+       * different retry policy, create a new connection
+       */
+      proxy3 = getClient(addr, newConf, anotherRetryPolicy);
+      proxy3.ping(null, newEmptyRequest());
+      assertEquals("number of connections in cache is wrong", 2, conns.size());
+    } finally {
+      server.stop();
+      // this is dirty, but clear out connection cache for next run
+      if (client != null) {
+        client.getConnectionIds().clear();
+      }
+      if (proxy1 != null) {
+        RPC.stopProxy(proxy1);
+      }
+      if (proxy2 != null) {
+        RPC.stopProxy(proxy2);
+      }
+      if (proxy3 != null) {
+        RPC.stopProxy(proxy3);
+      }
+    }
+  }
+}

+ 59 - 18
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java

@@ -30,18 +30,13 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.junit.Assert;
 
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.protobuf.TestProtos;
-import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos;
-import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.TokenSelector;
-import org.junit.Assert;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -112,7 +107,8 @@ public class TestRpcBase {
     return setupTestServer(builder);
   }
 
-  protected static RPC.Server setupTestServer(RPC.Builder builder) throws IOException {
+  protected static RPC.Server setupTestServer(
+      RPC.Builder builder) throws IOException {
     RPC.Server server = builder.build();
 
     server.start();
@@ -132,6 +128,24 @@ public class TestRpcBase {
     }
   }
 
+  protected static TestRpcService getClient(InetSocketAddress serverAddr,
+      Configuration clientConf, final RetryPolicy connectionRetryPolicy)
+      throws ServiceException {
+    try {
+      return RPC.getProtocolProxy(
+          TestRpcService.class,
+          0,
+          serverAddr,
+          UserGroupInformation.getCurrentUser(),
+          clientConf,
+          NetUtils.getDefaultSocketFactory(clientConf),
+          RPC.getRpcTimeout(clientConf),
+          connectionRetryPolicy, null).getProxy();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   protected static void stop(Server server, TestRpcService proxy) {
     if (proxy != null) {
       try {
@@ -175,17 +189,21 @@ public class TestRpcBase {
     public TestTokenIdentifier() {
       this(new Text(), new Text());
     }
+
     public TestTokenIdentifier(Text tokenid) {
       this(tokenid, new Text());
     }
+
     public TestTokenIdentifier(Text tokenid, Text realUser) {
       this.tokenid = tokenid == null ? new Text() : tokenid;
       this.realUser = realUser == null ? new Text() : realUser;
     }
+
     @Override
     public Text getKind() {
       return KIND_NAME;
     }
+
     @Override
     public UserGroupInformation getUser() {
       if (realUser.toString().isEmpty()) {
@@ -203,6 +221,7 @@ public class TestRpcBase {
       tokenid.readFields(in);
       realUser.readFields(in);
     }
+
     @Override
     public void write(DataOutput out) throws IOException {
       tokenid.write(out);
@@ -234,7 +253,7 @@ public class TestRpcBase {
     @SuppressWarnings("unchecked")
     @Override
     public Token<TestTokenIdentifier> selectToken(Text service,
-                                                  Collection<Token<? extends TokenIdentifier>> tokens) {
+                      Collection<Token<? extends TokenIdentifier>> tokens) {
       if (service == null) {
         return null;
       }
@@ -388,19 +407,17 @@ public class TestRpcBase {
     }
 
     @Override
-    public TestProtos.AuthUserResponseProto getAuthUser(
+    public TestProtos.UserResponseProto getAuthUser(
         RpcController controller, TestProtos.EmptyRequestProto request)
         throws ServiceException {
-      UserGroupInformation authUser = null;
+      UserGroupInformation authUser;
       try {
         authUser = UserGroupInformation.getCurrentUser();
       } catch (IOException e) {
         throw new ServiceException(e);
       }
 
-      return TestProtos.AuthUserResponseProto.newBuilder()
-          .setAuthUser(authUser.getUserName())
-          .build();
+      return newUserResponse(authUser.getUserName());
     }
 
     @Override
@@ -432,9 +449,37 @@ public class TestRpcBase {
 
       return TestProtos.EmptyResponseProto.newBuilder().build();
     }
+
+    @Override
+    public TestProtos.UserResponseProto getCurrentUser(
+        RpcController controller,
+        TestProtos.EmptyRequestProto request) throws ServiceException {
+      String user;
+      try {
+        user = UserGroupInformation.getCurrentUser().toString();
+      } catch (IOException e) {
+        throw new ServiceException("Failed to get current user", e);
+      }
+
+      return newUserResponse(user);
+    }
+
+    @Override
+    public TestProtos.UserResponseProto getServerRemoteUser(
+        RpcController controller,
+        TestProtos.EmptyRequestProto request) throws ServiceException {
+      String serverRemoteUser = Server.getRemoteUser().toString();
+      return newUserResponse(serverRemoteUser);
+    }
+
+    private TestProtos.UserResponseProto newUserResponse(String user) {
+      return TestProtos.UserResponseProto.newBuilder()
+          .setUser(user)
+          .build();
+    }
   }
 
-  protected static TestProtos.EmptyRequestProto newEmptyRequest() {
+  public static TestProtos.EmptyRequestProto newEmptyRequest() {
     return TestProtos.EmptyRequestProto.newBuilder().build();
   }
 
@@ -478,8 +523,4 @@ public class TestRpcBase {
     }
     return null;
   }
-
-  protected static String convert(TestProtos.AuthUserResponseProto response) {
-    return response.getAuthUser();
-  }
 }

+ 48 - 26
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java

@@ -45,30 +45,55 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import javax.security.auth.callback.*;
-import javax.security.sasl.*;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
 import java.io.IOException;
 import java.lang.annotation.Annotation;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.security.Security;
-import java.util.*;
-import java.util.concurrent.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
-import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.*;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.KERBEROS;
+import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.SIMPLE;
+import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.TOKEN;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /** Unit tests for using Sasl over RPC. */
 @RunWith(Parameterized.class)
 public class TestSaslRPC extends TestRpcBase {
   @Parameters
   public static Collection<Object[]> data() {
-    Collection<Object[]> params = new ArrayList<Object[]>();
+    Collection<Object[]> params = new ArrayList<>();
     for (QualityOfProtection qop : QualityOfProtection.values()) {
       params.add(new Object[]{ new QualityOfProtection[]{qop},qop, null });
     }
@@ -114,7 +139,7 @@ public class TestSaslRPC extends TestRpcBase {
     NONE(),
     VALID(),
     INVALID(),
-    OTHER();
+    OTHER()
   }
   
   @BeforeClass
@@ -230,7 +255,7 @@ public class TestSaslRPC extends TestRpcBase {
       final Server server = setupTestServer(conf, 5, sm);
       doDigestRpc(server, sm);
     } finally {
-      SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
+      SecurityUtil.setSecurityInfoProviders();
     }
   }
 
@@ -259,7 +284,7 @@ public class TestSaslRPC extends TestRpcBase {
     addr = NetUtils.getConnectAddress(server);
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId, sm);
+    Token<TestTokenIdentifier> token = new Token<>(tokenId, sm);
     SecurityUtil.setTokenService(token, addr);
     current.addToken(token);
 
@@ -296,8 +321,8 @@ public class TestSaslRPC extends TestRpcBase {
 
     // set doPing to true
     newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
-    ConnectionId remoteId = ConnectionId.getConnectionId(new InetSocketAddress(0),
-        TestRpcService.class, null, 0, null, newConf);
+    ConnectionId remoteId = ConnectionId.getConnectionId(
+        new InetSocketAddress(0), TestRpcService.class, null, 0, null, newConf);
     assertEquals(CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT,
         remoteId.getPingInterval());
     // set doPing to false
@@ -806,13 +831,13 @@ public class TestSaslRPC extends TestRpcBase {
     final TestTokenSecretManager sm = new TestTokenSecretManager();
     boolean useSecretManager = (serverAuth != SIMPLE);
     if (enableSecretManager != null) {
-      useSecretManager &= enableSecretManager.booleanValue();
+      useSecretManager &= enableSecretManager;
     }
     if (forceSecretManager != null) {
-      useSecretManager |= forceSecretManager.booleanValue();
+      useSecretManager |= forceSecretManager;
     }
     final SecretManager<?> serverSm = useSecretManager ? sm : null;
-    
+
     Server server = serverUgi.doAs(new PrivilegedExceptionAction<Server>() {
       @Override
       public Server run() throws IOException {
@@ -867,13 +892,13 @@ public class TestSaslRPC extends TestRpcBase {
             proxy.ping(null, newEmptyRequest());
             // make sure the other side thinks we are who we said we are!!!
             assertEquals(clientUgi.getUserName(),
-                convert(proxy.getAuthUser(null, newEmptyRequest())));
+                proxy.getAuthUser(null, newEmptyRequest()).getUser());
             AuthMethod authMethod =
                 convert(proxy.getAuthMethod(null, newEmptyRequest()));
             // verify sasl completed with correct QOP
             assertEquals((authMethod != SIMPLE) ? expectedQop.saslQop : null,
-                RPC.getConnectionIdForProxy(proxy).getSaslQop());
-            return authMethod.toString();
+                         RPC.getConnectionIdForProxy(proxy).getSaslQop());
+            return authMethod != null ? authMethod.toString() : null;
           } catch (ServiceException se) {
             if (se.getCause() instanceof RemoteException) {
               throw (RemoteException) se.getCause();
@@ -898,21 +923,18 @@ public class TestSaslRPC extends TestRpcBase {
       String actual) {
     assertEquals(expect.toString(), actual);
   }
-  
-  private static void assertAuthEquals(Pattern expect,
-      String actual) {
+
+  private static void assertAuthEquals(Pattern expect, String actual) {
     // this allows us to see the regexp and the value it didn't match
     if (!expect.matcher(actual).matches()) {
-      assertEquals(expect, actual); // it failed
-    } else {
-      assertTrue(true); // it matched
+      fail(); // it failed
     }
   }
 
   /*
    * Class used to test overriding QOP values using SaslPropertiesResolver
    */
-  static class AuthSaslPropertiesResolver extends SaslPropertiesResolver{
+  static class AuthSaslPropertiesResolver extends SaslPropertiesResolver {
 
     @Override
     public Map<String, String> getServerProperties(InetAddress address) {
@@ -921,7 +943,7 @@ public class TestSaslRPC extends TestRpcBase {
       return newPropertes;
     }
   }
-  
+
   public static void main(String[] args) throws Exception {
     System.out.println("Testing Kerberos authentication over RPC");
     if (args.length != 2) {

+ 103 - 188
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java

@@ -17,40 +17,35 @@
  */
 package org.apache.hadoop.security;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.NetworkInterface;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Enumeration;
-
-import org.junit.Assert;
-
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ipc.TestRpcBase;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.hadoop.ipc.TestRpcBase.TestTokenSecretManager;
-import org.apache.hadoop.ipc.TestRpcBase.TestTokenIdentifier;
-import org.apache.hadoop.ipc.TestRpcBase.TestTokenSelector;
-import org.apache.commons.logging.*;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Enumeration;
 
 /**
- *
+ * Test do as effective user.
  */
-public class TestDoAsEffectiveUser {
+public class TestDoAsEffectiveUser extends TestRpcBase {
   final private static String REAL_USER_NAME = "realUser1@HADOOP.APACHE.ORG";
   final private static String REAL_USER_SHORT_NAME = "realUser1";
   final private static String PROXY_USER_NAME = "proxyUser";
@@ -58,8 +53,8 @@ public class TestDoAsEffectiveUser {
   final private static String GROUP2_NAME = "group2";
   final private static String[] GROUP_NAMES = new String[] { GROUP1_NAME,
       GROUP2_NAME };
-  private static final String ADDRESS = "0.0.0.0";
-  private TestProtocol proxy;
+
+  private TestRpcService client;
   private static final Configuration masterConf = new Configuration();
   
   
@@ -82,7 +77,7 @@ public class TestDoAsEffectiveUser {
 
   private void configureSuperUserIPAddresses(Configuration conf,
       String superUserShortName) throws IOException {
-    ArrayList<String> ipList = new ArrayList<String>();
+    ArrayList<String> ipList = new ArrayList<>();
     Enumeration<NetworkInterface> netInterfaceList = NetworkInterface
         .getNetworkInterfaces();
     while (netInterfaceList.hasMoreElements()) {
@@ -130,50 +125,19 @@ public class TestDoAsEffectiveUser {
         curUGI.toString());
   }
 
-  @TokenInfo(TestTokenSelector.class)
-  public interface TestProtocol extends VersionedProtocol {
-    public static final long versionID = 1L;
-
-    String aMethod() throws IOException;
-    String getServerRemoteUser() throws IOException;
-  }
-
-  public class TestImpl implements TestProtocol {
-
-    @Override
-    public String aMethod() throws IOException {
-      return UserGroupInformation.getCurrentUser().toString();
-    }
-
-    @Override
-    public String getServerRemoteUser() throws IOException {
-      return Server.getRemoteUser().toString();
-    }
-    
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return TestProtocol.versionID;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      return new ProtocolSignature(TestProtocol.versionID, null);
-    }
-  }
-
-  private void checkRemoteUgi(final Server server,
-      final UserGroupInformation ugi, final Configuration conf)
-          throws Exception {
+  private void checkRemoteUgi(final UserGroupInformation ugi,
+                              final Configuration conf) throws Exception {
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
-      public Void run() throws IOException {
-        proxy = RPC.getProxy(
-            TestProtocol.class, TestProtocol.versionID,
-            NetUtils.getConnectAddress(server), conf);
-        Assert.assertEquals(ugi.toString(), proxy.aMethod());
-        Assert.assertEquals(ugi.toString(), proxy.getServerRemoteUser());
+      public Void run() throws ServiceException {
+        client = getClient(addr, conf);
+        String currentUser = client.getCurrentUser(null,
+            newEmptyRequest()).getUser();
+        String serverRemoteUser = client.getServerRemoteUser(null,
+            newEmptyRequest()).getUser();
+
+        Assert.assertEquals(ugi.toString(), currentUser);
+        Assert.assertEquals(ugi.toString(), serverRemoteUser);
         return null;
       }
     });    
@@ -185,29 +149,27 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
         getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME), "group1");
     configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).build();
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 5);
 
     refreshConf(conf);
     try {
-      server.start();
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
-      checkRemoteUgi(server, realUserUgi, conf);
+      checkRemoteUgi(realUserUgi, conf);
       
-      UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
+      UserGroupInformation proxyUserUgi =
+          UserGroupInformation.createProxyUserForTesting(
           PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      checkRemoteUgi(server, proxyUserUgi, conf);
+      checkRemoteUgi(proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -218,29 +180,25 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
             getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group1");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 5);
 
     refreshConf(conf);
     try {
-      server.start();
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
-      checkRemoteUgi(server, realUserUgi, conf);
+      checkRemoteUgi(realUserUgi, conf);
 
       UserGroupInformation proxyUserUgi = UserGroupInformation
           .createProxyUserForTesting(PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      checkRemoteUgi(server, proxyUserUgi, conf);
+      checkRemoteUgi(proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -256,17 +214,14 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
             getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group1");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 5);
 
     refreshConf(conf);
     
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -275,11 +230,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -287,10 +241,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
   
@@ -299,17 +250,14 @@ public class TestDoAsEffectiveUser {
     final Configuration conf = new Configuration();
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
         getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME), "group1");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 2);
 
     refreshConf(conf);
 
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -318,11 +266,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -330,10 +277,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -341,15 +285,12 @@ public class TestDoAsEffectiveUser {
   public void testRealUserGroupNotSpecified() throws IOException {
     final Configuration conf = new Configuration();
     configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 2);
 
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -358,11 +299,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = (TestProtocol) RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -370,10 +310,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
   
@@ -384,17 +321,14 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
             getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group3");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 2);
     
     refreshConf(conf);
 
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -403,11 +337,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -415,10 +348,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -432,20 +362,17 @@ public class TestDoAsEffectiveUser {
     final Configuration conf = new Configuration(masterConf);
     TestTokenSecretManager sm = new TestTokenSecretManager();
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
     UserGroupInformation.setConfiguration(conf);
-    final Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).setSecretManager(sm).build();
-
-    server.start();
+    final Server server = setupTestServer(conf, 5, sm);
 
     final UserGroupInformation current = UserGroupInformation
         .createRemoteUser(REAL_USER_NAME);    
-    
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()), new Text("SomeSuperUser"));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
+    Token<TestTokenIdentifier> token = new Token<>(tokenId,
         sm);
     SecurityUtil.setTokenService(token, addr);
     UserGroupInformation proxyUserUgi = UserGroupInformation
@@ -453,23 +380,19 @@ public class TestDoAsEffectiveUser {
     proxyUserUgi.addToken(token);
     
     refreshConf(conf);
-    
+
     String retVal = proxyUserUgi.doAs(new PrivilegedExceptionAction<String>() {
       @Override
       public String run() throws Exception {
         try {
-          proxy = RPC.getProxy(TestProtocol.class,
-              TestProtocol.versionID, addr, conf);
-          String ret = proxy.aMethod();
-          return ret;
+          client = getClient(addr, conf);
+          return client.getCurrentUser(null,
+              newEmptyRequest()).getUser();
         } catch (Exception e) {
           e.printStackTrace();
           throw e;
         } finally {
-          server.stop();
-          if (proxy != null) {
-            RPC.stopProxy(proxy);
-          }
+          stop(server, client);
         }
       }
     });
@@ -486,42 +409,34 @@ public class TestDoAsEffectiveUser {
     TestTokenSecretManager sm = new TestTokenSecretManager();
     final Configuration newConf = new Configuration(masterConf);
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, newConf);
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(newConf, TestRpcService.class,
+        ProtobufRpcEngine.class);
     UserGroupInformation.setConfiguration(newConf);
-    final Server server = new RPC.Builder(newConf)
-        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
-        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
-        .setSecretManager(sm).build();
-
-    server.start();
+    final Server server = setupTestServer(newConf, 5, sm);
 
     final UserGroupInformation current = UserGroupInformation
         .createUserForTesting(REAL_USER_NAME, GROUP_NAMES);
     
     refreshConf(newConf);
-    
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()), new Text("SomeSuperUser"));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
-        sm);
+    Token<TestTokenIdentifier> token = new Token<>(tokenId, sm);
     SecurityUtil.setTokenService(token, addr);
     current.addToken(token);
     String retVal = current.doAs(new PrivilegedExceptionAction<String>() {
       @Override
       public String run() throws Exception {
         try {
-          proxy = RPC.getProxy(TestProtocol.class,
-              TestProtocol.versionID, addr, newConf);
-          String ret = proxy.aMethod();
-          return ret;
+          client = getClient(addr, newConf);
+          return client.getCurrentUser(null,
+              newEmptyRequest()).getUser();
         } catch (Exception e) {
           e.printStackTrace();
           throw e;
         } finally {
-          server.stop();
-          if (proxy != null) {
-            RPC.stopProxy(proxy);
-          }
+          stop(server, client);
         }
       }
     });

+ 23 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.TestRpcBase.TestTokenIdentifier;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
@@ -28,7 +29,11 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.*;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
@@ -50,9 +55,22 @@ import java.util.Set;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL;
-import static org.apache.hadoop.ipc.TestSaslRPC.*;
-import static org.apache.hadoop.test.MetricsAsserts.*;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
+import static org.apache.hadoop.test.MetricsAsserts.assertGaugeGt;
+import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges;
+import static org.apache.hadoop.test.MetricsAsserts.getDoubleGauge;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -109,7 +127,7 @@ public class TestUserGroupInformation {
     UserGroupInformation.setLoginUser(null);
   }
 
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testSimpleLogin() throws IOException {
     tryLoginAuthenticationMethod(AuthenticationMethod.SIMPLE, true);
   }

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/proto/test.proto

@@ -88,6 +88,6 @@ message AuthMethodResponseProto {
   required string mechanismName = 2;
 }
 
-message AuthUserResponseProto {
-  required string authUser = 1;
+message UserResponseProto {
+  required string user = 1;
 }

+ 7 - 1
hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto

@@ -40,9 +40,11 @@ service TestProtobufRpcProto {
   rpc exchange(ExchangeRequestProto) returns (ExchangeResponseProto);
   rpc sleep(SleepRequestProto) returns (EmptyResponseProto);
   rpc getAuthMethod(EmptyRequestProto) returns (AuthMethodResponseProto);
-  rpc getAuthUser(EmptyRequestProto) returns (AuthUserResponseProto);
+  rpc getAuthUser(EmptyRequestProto) returns (UserResponseProto);
   rpc echoPostponed(EchoRequestProto) returns (EchoResponseProto);
   rpc sendPostponed(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc getCurrentUser(EmptyRequestProto) returns (UserResponseProto);
+  rpc getServerRemoteUser(EmptyRequestProto) returns (UserResponseProto);
 }
 
 service TestProtobufRpc2Proto {
@@ -65,3 +67,7 @@ service NewerProtobufRpcProto {
   rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
   rpc echo(EmptyRequestProto) returns (EmptyResponseProto);
 }
+
+service CustomProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+}

+ 32 - 10
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -143,23 +143,36 @@ class JsonUtilClient {
         storagePolicy, null);
   }
 
+  static HdfsFileStatus[] toHdfsFileStatusArray(final Map<?, ?> json) {
+    Preconditions.checkNotNull(json);
+    final Map<?, ?> rootmap =
+        (Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
+    final List<?> array = JsonUtilClient.getList(rootmap,
+        FileStatus.class.getSimpleName());
+
+    // convert FileStatus
+    Preconditions.checkNotNull(array);
+    final HdfsFileStatus[] statuses = new HdfsFileStatus[array.size()];
+    int i = 0;
+    for (Object object : array) {
+      final Map<?, ?> m = (Map<?, ?>) object;
+      statuses[i++] = JsonUtilClient.toFileStatus(m, false);
+    }
+    return statuses;
+  }
+
   static DirectoryListing toDirectoryListing(final Map<?, ?> json) {
     if (json == null) {
       return null;
     }
-    final List<?> list = JsonUtilClient.getList(json,
-        "partialListing");
+    final Map<?, ?> listing = getMap(json, "DirectoryListing");
+    final Map<?, ?> partialListing = getMap(listing, "partialListing");
+    HdfsFileStatus[] fileStatuses = toHdfsFileStatusArray(partialListing);
 
-    HdfsFileStatus[] partialListing = new HdfsFileStatus[list.size()];
-    int i = 0;
-    for (Object o : list) {
-      final Map<?, ?> m = (Map<?, ?>) o;
-      partialListing[i++] = toFileStatus(m, false);
-    }
-    int remainingEntries = getInt(json, "remainingEntries", -1);
+    int remainingEntries = getInt(listing, "remainingEntries", -1);
     Preconditions.checkState(remainingEntries != -1,
         "remainingEntries was not set");
-    return new DirectoryListing(partialListing, remainingEntries);
+    return new DirectoryListing(fileStatuses, remainingEntries);
   }
 
   /** Convert a Json map to an ExtendedBlock object. */
@@ -210,6 +223,15 @@ class JsonUtilClient {
     }
   }
 
+  static Map<?, ?> getMap(Map<?, ?> m, String key) {
+    Object map = m.get(key);
+    if (map instanceof Map<?, ?>) {
+      return (Map<?, ?>) map;
+    } else {
+      return null;
+    }
+  }
+
   /** Convert a Json map to an DatanodeInfo object. */
   static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
       throws IOException {

+ 27 - 60
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -64,7 +64,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
@@ -1491,75 +1490,43 @@ public class WebHdfsFileSystem extends FileSystem
     return new FsPathResponseRunner<FileStatus[]>(op, f) {
       @Override
       FileStatus[] decodeResponse(Map<?,?> json) {
-        final Map<?, ?> rootmap =
-            (Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
-        final List<?> array = JsonUtilClient.getList(rootmap,
-            FileStatus.class.getSimpleName());
-
-        //convert FileStatus
-        assert array != null;
-        final FileStatus[] statuses = new FileStatus[array.size()];
-        int i = 0;
-        for (Object object : array) {
-          final Map<?, ?> m = (Map<?, ?>) object;
-          statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false),
-              f);
+        HdfsFileStatus[] hdfsStatuses =
+            JsonUtilClient.toHdfsFileStatusArray(json);
+        final FileStatus[] statuses = new FileStatus[hdfsStatuses.length];
+        for (int i = 0; i < hdfsStatuses.length; i++) {
+          statuses[i] = makeQualified(hdfsStatuses[i], f);
         }
+
         return statuses;
       }
     }.run();
   }
 
   private static final byte[] EMPTY_ARRAY = new byte[] {};
-  private class DirListingIterator<T extends FileStatus> implements
-      RemoteIterator<T> {
-
-    private final Path path;
-    private DirectoryListing thisListing;
-    private int i = 0;
-    private byte[] prevKey = EMPTY_ARRAY;
 
-    DirListingIterator(Path path) {
-      this.path = path;
-    }
-
-    @Override
-    public boolean hasNext() throws IOException {
-      if (thisListing == null) {
-        fetchMore();
-      }
-      return i < thisListing.getPartialListing().length ||
-          thisListing.hasMore();
-    }
-
-    private void fetchMore() throws IOException {
-      thisListing = new FsPathResponseRunner<DirectoryListing>(
-          GetOpParam.Op.LISTSTATUS_BATCH,
-          path, new StartAfterParam(new String(prevKey, Charsets.UTF_8))) {
-        @Override
-        DirectoryListing decodeResponse(Map<?, ?> json) throws IOException {
-          return JsonUtilClient.toDirectoryListing(json);
-        }
-      }.run();
-      i = 0;
-      prevKey = thisListing.getLastName();
+  @Override
+  public DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+      FileNotFoundException, IOException {
+    byte[] prevKey = EMPTY_ARRAY;
+    if (token != null) {
+      prevKey = token;
     }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public T next() throws IOException {
-      Preconditions.checkState(hasNext(), "No more items in iterator");
-      if (i == thisListing.getPartialListing().length) {
-        fetchMore();
+    DirectoryListing listing = new FsPathResponseRunner<DirectoryListing>(
+        GetOpParam.Op.LISTSTATUS_BATCH,
+        f, new StartAfterParam(new String(prevKey, Charsets.UTF_8))) {
+      @Override
+      DirectoryListing decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtilClient.toDirectoryListing(json);
       }
-      return (T)makeQualified(thisListing.getPartialListing()[i++], path);
-    }
-  }
-
-  @Override
-  public RemoteIterator<FileStatus> listStatusIterator(final Path f)
-      throws FileNotFoundException, IOException {
-    return new DirListingIterator<>(f);
+    }.run();
+    // Qualify the returned FileStatus array
+    final HdfsFileStatus[] statuses = listing.getPartialListing();
+    FileStatus[] qualified = new FileStatus[statuses.length];
+    for (int i = 0; i < statuses.length; i++) {
+      qualified[i] = makeQualified(statuses[i], f);
+    }
+    return new DirectoryEntries(qualified, listing.getLastName(),
+        listing.hasMore());
   }
 
   @Override

+ 48 - 8
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.http.client;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
+
+import com.google.common.base.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
@@ -111,6 +113,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String XATTR_SET_FLAG_PARAM = "flag";
   public static final String XATTR_ENCODING_PARAM = "encoding";
   public static final String NEW_LENGTH_PARAM = "newlength";
+  public static final String START_AFTER_PARAM = "startAfter";
 
   public static final Short DEFAULT_PERMISSION = 0755;
   public static final String ACLSPEC_DEFAULT = "";
@@ -184,6 +187,10 @@ public class HttpFSFileSystem extends FileSystem
 
   public static final String ENC_BIT_JSON = "encBit";
 
+  public static final String DIRECTORY_LISTING_JSON = "DirectoryListing";
+  public static final String PARTIAL_LISTING_JSON = "partialListing";
+  public static final String REMAINING_ENTRIES_JSON = "remainingEntries";
+
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
   private static final String HTTP_GET = "GET";
@@ -203,7 +210,7 @@ public class HttpFSFileSystem extends FileSystem
     MODIFYACLENTRIES(HTTP_PUT), REMOVEACLENTRIES(HTTP_PUT),
     REMOVEDEFAULTACL(HTTP_PUT), REMOVEACL(HTTP_PUT), SETACL(HTTP_PUT),
     DELETE(HTTP_DELETE), SETXATTR(HTTP_PUT), GETXATTRS(HTTP_GET),
-    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET);
+    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET), LISTSTATUS_BATCH(HTTP_GET);
 
     private String httpMethod;
 
@@ -666,6 +673,17 @@ public class HttpFSFileSystem extends FileSystem
     return (Boolean) json.get(DELETE_JSON);
   }
 
+  private FileStatus[] toFileStatuses(JSONObject json, Path f) {
+    json = (JSONObject) json.get(FILE_STATUSES_JSON);
+    JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
+    FileStatus[] array = new FileStatus[jsonArray.size()];
+    f = makeQualified(f);
+    for (int i = 0; i < jsonArray.size(); i++) {
+      array[i] = createFileStatus(f, (JSONObject) jsonArray.get(i));
+    }
+    return array;
+  }
+
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
@@ -684,14 +702,36 @@ public class HttpFSFileSystem extends FileSystem
                                            params, f, true);
     HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
-    json = (JSONObject) json.get(FILE_STATUSES_JSON);
-    JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
-    FileStatus[] array = new FileStatus[jsonArray.size()];
-    f = makeQualified(f);
-    for (int i = 0; i < jsonArray.size(); i++) {
-      array[i] = createFileStatus(f, (JSONObject) jsonArray.get(i));
+    return toFileStatuses(json, f);
+  }
+
+  @Override
+  public DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+      FileNotFoundException, IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.LISTSTATUS_BATCH.toString());
+    if (token != null) {
+      params.put(START_AFTER_PARAM, new String(token, Charsets.UTF_8));
     }
-    return array;
+    HttpURLConnection conn = getConnection(
+        Operation.LISTSTATUS_BATCH.getMethod(),
+        params, f, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    // Parse the FileStatus array
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    JSONObject listing = (JSONObject) json.get(DIRECTORY_LISTING_JSON);
+    FileStatus[] statuses = toFileStatuses(
+        (JSONObject) listing.get(PARTIAL_LISTING_JSON), f);
+    // New token is the last FileStatus entry
+    byte[] newToken = null;
+    if (statuses.length > 0) {
+      newToken = statuses[statuses.length - 1].getPath().getName().toString()
+          .getBytes(Charsets.UTF_8);
+    }
+    // Parse the remainingEntries boolean into hasMore
+    final long remainingEntries = (Long) listing.get(REMAINING_ENTRIES_JSON);
+    final boolean hasMore = remainingEntries > 0 ? true : false;
+    return new DirectoryEntries(statuses, newToken, hasMore);
   }
 
   /**

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java

@@ -43,6 +43,8 @@ public class HttpFSUtils {
 
   public static final String SERVICE_VERSION = "/v1";
 
+  public static final byte[] EMPTY_BYTES = {};
+
   private static final String SERVICE_PATH = SERVICE_NAME + SERVICE_VERSION;
 
   /**

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

@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.GlobFilter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
@@ -37,6 +38,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -108,6 +110,27 @@ public class FSOperations {
     return json;
   }
 
+  /**
+   * Serializes a DirectoryEntries object into the JSON for a
+   * WebHDFS {@link org.apache.hadoop.hdfs.protocol.DirectoryListing}.
+   * <p>
+   * These two classes are slightly different, due to the impedance
+   * mismatches between the WebHDFS and FileSystem APIs.
+   * @param entries
+   * @return json
+   */
+  private static Map<String, Object> toJson(FileSystem.DirectoryEntries
+      entries) {
+    Map<String, Object> json = new LinkedHashMap<>();
+    Map<String, Object> inner = new LinkedHashMap<>();
+    Map<String, Object> fileStatuses = toJson(entries.getEntries());
+    inner.put(HttpFSFileSystem.PARTIAL_LISTING_JSON, fileStatuses);
+    inner.put(HttpFSFileSystem.REMAINING_ENTRIES_JSON, entries.hasMore() ? 1
+        : 0);
+    json.put(HttpFSFileSystem.DIRECTORY_LISTING_JSON, inner);
+    return json;
+  }
+
   /** Converts an <code>AclStatus</code> object into a JSON object.
    *
    * @param aclStatus AclStatus object
@@ -624,6 +647,45 @@ public class FSOperations {
 
   }
 
+  /**
+   * Executor that performs a batched directory listing.
+   */
+  @InterfaceAudience.Private
+  public static class FSListStatusBatch implements FileSystemAccess
+      .FileSystemExecutor<Map> {
+    private final Path path;
+    private final byte[] token;
+
+    public FSListStatusBatch(String path, byte[] token) throws IOException {
+      this.path = new Path(path);
+      this.token = token.clone();
+    }
+
+    /**
+     * Simple wrapper filesystem that exposes the protected batched
+     * listStatus API so we can use it.
+     */
+    private static class WrappedFileSystem extends FilterFileSystem {
+      public WrappedFileSystem(FileSystem f) {
+        super(f);
+      }
+
+      @Override
+      public DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+          FileNotFoundException, IOException {
+        return super.listStatusBatch(f, token);
+      }
+    }
+
+    @Override
+    public Map execute(FileSystem fs) throws IOException {
+      WrappedFileSystem wrappedFS = new WrappedFileSystem(fs);
+      FileSystem.DirectoryEntries entries =
+          wrappedFS.listStatusBatch(path, token);
+      return toJson(entries);
+    }
+  }
+
   /**
    * Executor that performs a mkdirs FileSystemAccess files system operation.
    */

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -91,6 +91,8 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.GETXATTRS, 
         new Class[]{XAttrNameParam.class, XAttrEncodingParam.class});
     PARAMS_DEF.put(Operation.LISTXATTRS, new Class[]{});
+    PARAMS_DEF.put(Operation.LISTSTATUS_BATCH,
+        new Class[]{StartAfterParam.class});
   }
 
   public HttpFSParametersProvider() {
@@ -520,4 +522,22 @@ public class HttpFSParametersProvider extends ParametersProvider {
       super(NAME, XAttrCodec.class, null);
     }
   }
+
+  /**
+   * Class for startafter parameter.
+   */
+  @InterfaceAudience.Private
+  public static class StartAfterParam extends StringParam {
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.START_AFTER_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public StartAfterParam() {
+      super(NAME, null);
+    }
+  }
 }

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java

@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.fs.http.server;
 
+import com.google.common.base.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.fs.http.client.HttpFSUtils;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AccessTimeParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AclPermissionParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.BlockSizeParam;
@@ -320,6 +322,21 @@ public class HttpFSServer {
       response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
       break;
     }
+    case LISTSTATUS_BATCH: {
+      String startAfter = params.get(
+          HttpFSParametersProvider.StartAfterParam.NAME,
+          HttpFSParametersProvider.StartAfterParam.class);
+      byte[] token = HttpFSUtils.EMPTY_BYTES;
+      if (startAfter != null) {
+        token = startAfter.getBytes(Charsets.UTF_8);
+      }
+      FSOperations.FSListStatusBatch command = new FSOperations
+          .FSListStatusBatch(path, token);
+      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}] token [{}]", path, token);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
     default: {
       throw new IOException(
           MessageFormat.format("Invalid HTTP GET operation [{0}]", op.value()));

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

@@ -84,7 +84,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
       count = 0;
     }
 
-    synchronized FileSystem getFileSytem(Configuration conf)
+    synchronized FileSystem getFileSystem(Configuration conf)
       throws IOException {
       if (fs == null) {
         fs = FileSystem.get(conf);
@@ -290,7 +290,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     }
     Configuration conf = new Configuration(namenodeConf);
     conf.set(HTTPFS_FS_USER, user);
-    return cachedFS.getFileSytem(conf);
+    return cachedFS.getFileSystem(conf);
   }
 
   protected void closeFileSystem(FileSystem fs) throws IOException {

+ 59 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -44,6 +45,7 @@ import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.test.TestJetty;
 import org.apache.hadoop.test.TestJettyHelper;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -62,6 +64,7 @@ import java.io.Writer;
 import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -136,14 +139,19 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     return "webhdfs";
   }
 
-  protected FileSystem getHttpFSFileSystem() throws Exception {
-    Configuration conf = new Configuration();
+  protected FileSystem getHttpFSFileSystem(Configuration conf) throws
+      Exception {
     conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
     URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     return FileSystem.get(uri, conf);
   }
 
+  protected FileSystem getHttpFSFileSystem() throws Exception {
+    Configuration conf = new Configuration();
+    return getHttpFSFileSystem(conf);
+  }
+
   protected void testGet() throws Exception {
     FileSystem fs = getHttpFSFileSystem();
     Assert.assertNotNull(fs);
@@ -355,6 +363,51 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     assertEquals(stati[0].getPath().getName(), path.getName());
   }
 
+  private static void assertSameListing(FileSystem expected, FileSystem
+      actual, Path p) throws IOException {
+    // Consume all the entries from both iterators
+    RemoteIterator<FileStatus> exIt = expected.listStatusIterator(p);
+    List<FileStatus> exStatuses = new ArrayList<>();
+    while (exIt.hasNext()) {
+      exStatuses.add(exIt.next());
+    }
+    RemoteIterator<FileStatus> acIt = actual.listStatusIterator(p);
+    List<FileStatus> acStatuses = new ArrayList<>();
+    while (acIt.hasNext()) {
+      acStatuses.add(acIt.next());
+    }
+    assertEquals(exStatuses.size(), acStatuses.size());
+    for (int i = 0; i < exStatuses.size(); i++) {
+      FileStatus expectedStatus = exStatuses.get(i);
+      FileStatus actualStatus = acStatuses.get(i);
+      // Path URIs are fully qualified, so compare just the path component
+      assertEquals(expectedStatus.getPath().toUri().getPath(),
+          actualStatus.getPath().toUri().getPath());
+    }
+  }
+
+  private void testListStatusBatch() throws Exception {
+    // LocalFileSystem writes checksum files next to the data files, which
+    // show up when listing via LFS. This makes the listings not compare
+    // properly.
+    Assume.assumeFalse(isLocalFS());
+
+    FileSystem proxyFs = FileSystem.get(getProxiedFSConf());
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 2);
+    FileSystem httpFs = getHttpFSFileSystem(conf);
+
+    // Test an empty directory
+    Path dir = new Path(getProxiedFSTestDir(), "dir");
+    proxyFs.mkdirs(dir);
+    assertSameListing(proxyFs, httpFs, dir);
+    // Create and test in a loop
+    for (int i = 0; i < 10; i++) {
+      proxyFs.create(new Path(dir, "file" + i)).close();
+      assertSameListing(proxyFs, httpFs, dir);
+    }
+  }
+
   private void testWorkingdirectory() throws Exception {
     FileSystem fs = FileSystem.get(getProxiedFSConf());
     Path workingDir = fs.getWorkingDirectory();
@@ -863,7 +916,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     GET, OPEN, CREATE, APPEND, TRUNCATE, CONCAT, RENAME, DELETE, LIST_STATUS, 
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER, 
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
-    GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION
+    GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH
   }
 
   private void operation(Operation op) throws Exception {
@@ -940,6 +993,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       case ENCRYPTION:
         testEncryption();
         break;
+      case LIST_STATUS_BATCH:
+        testListStatusBatch();
+        break;
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -814,7 +814,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle),
           iug);
       if (readCount < count) {
-        LOG.info("Partical read. Asked offset: " + offset + " count: " + count
+        LOG.info("Partial read. Asked offset: " + offset + " count: " + count
             + " and read back: " + readCount + " file size: "
             + attrs.getSize());
       }

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -741,7 +742,20 @@ public class BlockPoolSliceStorage extends Storage {
    *
    * @return the trash directory for a given block file that is being deleted.
    */
-  public String getTrashDirectory(File blockFile) {
+  public String getTrashDirectory(ReplicaInfo info) {
+
+    URI blockURI = info.getBlockURI();
+    try{
+      File blockFile = new File(blockURI);
+      return getTrashDirectory(blockFile);
+    } catch (IllegalArgumentException e) {
+      LOG.warn("Failed to get block file for replica " + info, e);
+    }
+
+    return null;
+  }
+
+  private String getTrashDirectory(File blockFile) {
     if (isTrashAllowed(blockFile)) {
       Matcher matcher = BLOCK_POOL_CURRENT_PATH_PATTERN.matcher(blockFile.getParent());
       String trashDirectory = matcher.replaceFirst("$1$2" + TRASH_ROOT_DIR + "$4");

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

@@ -121,7 +121,7 @@ class BlockReceiver implements Closeable {
   /** the block to receive */
   private final ExtendedBlock block; 
   /** the replica to write */
-  private ReplicaInPipelineInterface replicaInfo;
+  private ReplicaInPipeline replicaInfo;
   /** pipeline stage */
   private final BlockConstructionStage stage;
   private final boolean isTransfer;

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

@@ -165,6 +165,10 @@ public class BlockScanner {
     }
   }
 
+  public BlockScanner(DataNode datanode) {
+    this(datanode, datanode.getConf());
+  }
+
   public BlockScanner(DataNode datanode, Configuration conf) {
     this.datanode = datanode;
     this.conf = new Conf(conf);

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

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@@ -248,8 +249,8 @@ class BlockSender implements java.io.Closeable {
       }
       // if there is a write in progress
       ChunkChecksum chunkChecksum = null;
-      if (replica instanceof ReplicaBeingWritten) {
-        final ReplicaBeingWritten rbw = (ReplicaBeingWritten)replica;
+      if (replica.getState() == ReplicaState.RBW) {
+        final ReplicaInPipeline rbw = (ReplicaInPipeline) replica;
         waitForMinLength(rbw, startOffset + length);
         chunkChecksum = rbw.getLastChecksumAndDataLen();
       }
@@ -473,7 +474,7 @@ class BlockSender implements java.io.Closeable {
    * @param len minimum length to reach
    * @throws IOException on failing to reach the len in given wait time
    */
-  private static void waitForMinLength(ReplicaBeingWritten rbw, long len)
+  private static void waitForMinLength(ReplicaInPipeline rbw, long len)
       throws IOException {
     // Wait for 3 seconds for rbw replica to reach the minimum length
     for (int i = 0; i < 30 && rbw.getBytesOnDisk() < len; i++) {

+ 51 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java

@@ -56,6 +56,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.IGNORE_SECURE_PORTS_FOR_TESTI
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT;
 
+import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -72,7 +73,6 @@ import java.util.concurrent.TimeUnit;
  */
 @InterfaceAudience.Private
 public class DNConf {
-  final Configuration conf;
   final int socketTimeout;
   final int socketWriteTimeout;
   final int socketKeepaliveTimeout;
@@ -120,73 +120,77 @@ public class DNConf {
   private final int volFailuresTolerated;
   private final int volsConfigured;
   private final int maxDataLength;
+  private Configurable dn;
 
-  public DNConf(Configuration conf) {
-    this.conf = conf;
-    socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+  public DNConf(final Configurable dn) {
+    this.dn = dn;
+    socketTimeout = getConf().getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
         HdfsConstants.READ_TIMEOUT);
-    socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
+    socketWriteTimeout = getConf().getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
         HdfsConstants.WRITE_TIMEOUT);
-    socketKeepaliveTimeout = conf.getInt(
+    socketKeepaliveTimeout = getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY,
         DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
-    this.transferSocketSendBufferSize = conf.getInt(
+    this.transferSocketSendBufferSize = getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_KEY,
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_DEFAULT);
-    this.transferSocketRecvBufferSize = conf.getInt(
+    this.transferSocketRecvBufferSize = getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_KEY,
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_DEFAULT);
-    this.tcpNoDelay = conf.getBoolean(
+    this.tcpNoDelay = getConf().getBoolean(
         DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY,
         DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY_DEFAULT);
 
     /* Based on results on different platforms, we might need set the default
      * to false on some of them. */
-    transferToAllowed = conf.getBoolean(
+    transferToAllowed = getConf().getBoolean(
         DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
         DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT);
 
-    writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 
+    writePacketSize = getConf().getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
         DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
     
-    readaheadLength = conf.getLong(
+    readaheadLength = getConf().getLong(
         HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY,
         HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
-    maxDataLength = conf.getInt(DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH,
+    maxDataLength = getConf().getInt(DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH,
         DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
-    dropCacheBehindWrites = conf.getBoolean(
+    dropCacheBehindWrites = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY,
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT);
-    syncBehindWrites = conf.getBoolean(
+    syncBehindWrites = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_KEY,
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT);
-    syncBehindWritesInBackground = conf.getBoolean(
+    syncBehindWritesInBackground = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_KEY,
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_DEFAULT);
-    dropCacheBehindReads = conf.getBoolean(
+    dropCacheBehindReads = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY,
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT);
-    connectToDnViaHostname = conf.getBoolean(
+    connectToDnViaHostname = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
-    this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
+    this.blockReportInterval = getConf().getLong(
+        DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
         DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
-    this.ibrInterval = conf.getLong(
+    this.ibrInterval = getConf().getLong(
         DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY,
         DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT);
-    this.blockReportSplitThreshold = conf.getLong(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
-                                            DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
-    this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
+    this.blockReportSplitThreshold = getConf().getLong(
+        DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
+        DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
+    this.cacheReportInterval = getConf().getLong(
+        DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
         DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT);
 
-    this.dfsclientSlowIoWarningThresholdMs = conf.getLong(
+    this.dfsclientSlowIoWarningThresholdMs = getConf().getLong(
         HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
         HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
-    this.datanodeSlowIoWarningThresholdMs = conf.getLong(
+    this.datanodeSlowIoWarningThresholdMs = getConf().getLong(
         DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY,
         DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
 
-    long initBRDelay = conf.getTimeDuration(
+    long initBRDelay = getConf().getTimeDuration(
         DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
         DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT, TimeUnit.SECONDS) * 1000L;
     if (initBRDelay >= blockReportInterval) {
@@ -197,11 +201,11 @@ public class DNConf {
     }
     initialBlockReportDelayMs = initBRDelay;
     
-    heartBeatInterval = conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
+    heartBeatInterval = getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000L;
     long confLifelineIntervalMs =
-        conf.getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
-        3 * conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
+        getConf().getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
+        3 * getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS)) * 1000L;
     if (confLifelineIntervalMs <= heartBeatInterval) {
       confLifelineIntervalMs = 3 * heartBeatInterval;
@@ -215,47 +219,50 @@ public class DNConf {
     lifelineIntervalMs = confLifelineIntervalMs;
     
     // do we need to sync block file contents to disk when blockfile is closed?
-    this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
+    this.syncOnClose = getConf().getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY,
         DFS_DATANODE_SYNCONCLOSE_DEFAULT);
 
-    this.minimumNameNodeVersion = conf.get(DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY,
+    this.minimumNameNodeVersion = getConf().get(
+        DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY,
         DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT);
     
-    this.encryptDataTransfer = conf.getBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY,
+    this.encryptDataTransfer = getConf().getBoolean(
+        DFS_ENCRYPT_DATA_TRANSFER_KEY,
         DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
-    this.encryptionAlgorithm = conf.get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
-    this.trustedChannelResolver = TrustedChannelResolver.getInstance(conf);
+    this.encryptionAlgorithm = getConf().get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
+    this.trustedChannelResolver = TrustedChannelResolver.getInstance(getConf());
     this.saslPropsResolver = DataTransferSaslUtil.getSaslPropertiesResolver(
-      conf);
-    this.ignoreSecurePortsForTesting = conf.getBoolean(
+      getConf());
+    this.ignoreSecurePortsForTesting = getConf().getBoolean(
         IGNORE_SECURE_PORTS_FOR_TESTING_KEY,
         IGNORE_SECURE_PORTS_FOR_TESTING_DEFAULT);
     
-    this.xceiverStopTimeout = conf.getLong(
+    this.xceiverStopTimeout = getConf().getLong(
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT);
 
-    this.maxLockedMemory = conf.getLong(
+    this.maxLockedMemory = getConf().getLong(
         DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
         DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);
 
-    this.restartReplicaExpiry = conf.getLong(
+    this.restartReplicaExpiry = getConf().getLong(
         DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY,
         DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT) * 1000L;
 
-    this.allowNonLocalLazyPersist = conf.getBoolean(
+    this.allowNonLocalLazyPersist = getConf().getBoolean(
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST,
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST_DEFAULT);
 
-    this.bpReadyTimeout = conf.getTimeDuration(
+    this.bpReadyTimeout = getConf().getTimeDuration(
         DFS_DATANODE_BP_READY_TIMEOUT_KEY,
         DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT, TimeUnit.SECONDS);
 
     this.volFailuresTolerated =
-        conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
+        getConf().getInt(
+            DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
             DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
     String[] dataDirs =
-        conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+        getConf().getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
     this.volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
   }
 
@@ -270,7 +277,7 @@ public class DNConf {
    * @return Configuration the configuration
    */
   public Configuration getConf() {
-    return conf;
+    return this.dn.getConf();
   }
 
   /**

+ 68 - 67
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -363,7 +363,6 @@ public class DataNode extends ReconfigurableBase
   private SecureResources secureResources = null;
   // dataDirs must be accessed while holding the DataNode lock.
   private List<StorageLocation> dataDirs;
-  private Configuration conf;
   private final String confVersion;
   private final long maxNumberOfBlocksToLog;
   private final boolean pipelineSupportECN;
@@ -419,7 +418,7 @@ public class DataNode extends ReconfigurableBase
     this.confVersion = null;
     this.usersWithLocalPathAccess = null;
     this.connectToDnViaHostname = false;
-    this.blockScanner = new BlockScanner(this, conf);
+    this.blockScanner = new BlockScanner(this, this.getConf());
     this.pipelineSupportECN = false;
     this.checkDiskErrorInterval =
         ThreadLocalRandom.current().nextInt(5000, (int) (5000 * 1.25));
@@ -438,7 +437,7 @@ public class DataNode extends ReconfigurableBase
     this.tracer = createTracer(conf);
     this.tracerConfigurationManager =
         new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
-    this.blockScanner = new BlockScanner(this, conf);
+    this.blockScanner = new BlockScanner(this);
     this.lastDiskErrorCheck = 0;
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
         DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
@@ -487,7 +486,7 @@ public class DataNode extends ReconfigurableBase
     try {
       hostName = getHostName(conf);
       LOG.info("Configured hostname is " + hostName);
-      startDataNode(conf, dataDirs, resources);
+      startDataNode(dataDirs, resources);
     } catch (IOException ie) {
       shutdown();
       throw ie;
@@ -527,7 +526,7 @@ public class DataNode extends ReconfigurableBase
         try {
           LOG.info("Reconfiguring " + property + " to " + newVal);
           this.refreshVolumes(newVal);
-          return conf.get(DFS_DATANODE_DATA_DIR_KEY);
+          return getConf().get(DFS_DATANODE_DATA_DIR_KEY);
         } catch (IOException e) {
           rootException = e;
         } finally {
@@ -650,7 +649,7 @@ public class DataNode extends ReconfigurableBase
 
     // Use the existing StorageLocation to detect storage type changes.
     Map<String, StorageLocation> existingLocations = new HashMap<>();
-    for (StorageLocation loc : getStorageLocations(this.conf)) {
+    for (StorageLocation loc : getStorageLocations(getConf())) {
       existingLocations.put(loc.getFile().getCanonicalPath(), loc);
     }
 
@@ -846,7 +845,7 @@ public class DataNode extends ReconfigurableBase
         it.remove();
       }
     }
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, Joiner.on(",").join(dataDirs));
+    getConf().set(DFS_DATANODE_DATA_DIR_KEY, Joiner.on(",").join(dataDirs));
 
     if (ioe != null) {
       throw ioe;
@@ -904,14 +903,14 @@ public class DataNode extends ReconfigurableBase
    * for information related to the different configuration options and
    * Http Policy is decided.
    */
-  private void startInfoServer(Configuration conf)
+  private void startInfoServer()
     throws IOException {
     // SecureDataNodeStarter will bind the privileged port to the channel if
     // the DN is started by JSVC, pass it along.
     ServerSocketChannel httpServerChannel = secureResources != null ?
         secureResources.getHttpServerChannel() : null;
 
-    this.httpServer = new DatanodeHttpServer(conf, this, httpServerChannel);
+    httpServer = new DatanodeHttpServer(getConf(), this, httpServerChannel);
     httpServer.start();
     if (httpServer.getHttpAddress() != null) {
       infoPort = httpServer.getHttpAddress().getPort();
@@ -933,24 +932,24 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
-  private void initIpcServer(Configuration conf) throws IOException {
+  private void initIpcServer() throws IOException {
     InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
-        conf.getTrimmed(DFS_DATANODE_IPC_ADDRESS_KEY));
+        getConf().getTrimmed(DFS_DATANODE_IPC_ADDRESS_KEY));
     
     // Add all the RPC protocols that the Datanode implements    
-    RPC.setProtocolEngine(conf, ClientDatanodeProtocolPB.class,
+    RPC.setProtocolEngine(getConf(), ClientDatanodeProtocolPB.class,
         ProtobufRpcEngine.class);
     ClientDatanodeProtocolServerSideTranslatorPB clientDatanodeProtocolXlator = 
           new ClientDatanodeProtocolServerSideTranslatorPB(this);
     BlockingService service = ClientDatanodeProtocolService
         .newReflectiveBlockingService(clientDatanodeProtocolXlator);
-    ipcServer = new RPC.Builder(conf)
+    ipcServer = new RPC.Builder(getConf())
         .setProtocol(ClientDatanodeProtocolPB.class)
         .setInstance(service)
         .setBindAddress(ipcAddr.getHostName())
         .setPort(ipcAddr.getPort())
         .setNumHandlers(
-            conf.getInt(DFS_DATANODE_HANDLER_COUNT_KEY,
+            getConf().getInt(DFS_DATANODE_HANDLER_COUNT_KEY,
                 DFS_DATANODE_HANDLER_COUNT_DEFAULT)).setVerbose(false)
         .setSecretManager(blockPoolTokenSecretManager).build();
 
@@ -958,29 +957,32 @@ public class DataNode extends ReconfigurableBase
         = new ReconfigurationProtocolServerSideTranslatorPB(this);
     service = ReconfigurationProtocolService
         .newReflectiveBlockingService(reconfigurationProtocolXlator);
-    DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class, service,
+    DFSUtil.addPBProtocol(getConf(), ReconfigurationProtocolPB.class, service,
         ipcServer);
 
     InterDatanodeProtocolServerSideTranslatorPB interDatanodeProtocolXlator = 
         new InterDatanodeProtocolServerSideTranslatorPB(this);
     service = InterDatanodeProtocolService
         .newReflectiveBlockingService(interDatanodeProtocolXlator);
-    DFSUtil.addPBProtocol(conf, InterDatanodeProtocolPB.class, service,
+    DFSUtil.addPBProtocol(getConf(), InterDatanodeProtocolPB.class, service,
         ipcServer);
 
     TraceAdminProtocolServerSideTranslatorPB traceAdminXlator =
         new TraceAdminProtocolServerSideTranslatorPB(this);
     BlockingService traceAdminService = TraceAdminService
         .newReflectiveBlockingService(traceAdminXlator);
-    DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class, traceAdminService,
+    DFSUtil.addPBProtocol(
+        getConf(),
+        TraceAdminProtocolPB.class,
+        traceAdminService,
         ipcServer);
 
     LOG.info("Opened IPC server at " + ipcServer.getListenerAddress());
 
     // set service-level authorization security policy
-    if (conf.getBoolean(
+    if (getConf().getBoolean(
         CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
-      ipcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
+      ipcServer.refreshServiceAcl(getConf(), new HDFSPolicyProvider());
     }
   }
 
@@ -1071,17 +1073,17 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
-  private void initDataXceiver(Configuration conf) throws IOException {
+  private void initDataXceiver() throws IOException {
     // find free port or use privileged port provided
     TcpPeerServer tcpPeerServer;
     if (secureResources != null) {
       tcpPeerServer = new TcpPeerServer(secureResources);
     } else {
-      int backlogLength = conf.getInt(
+      int backlogLength = getConf().getInt(
           CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_KEY,
           CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT);
       tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
-          DataNode.getStreamingAddr(conf), backlogLength);
+          DataNode.getStreamingAddr(getConf()), backlogLength);
     }
     if (dnConf.getTransferSocketRecvBufferSize() > 0) {
       tcpPeerServer.setReceiveBufferSize(
@@ -1090,24 +1092,27 @@ public class DataNode extends ReconfigurableBase
     streamingAddr = tcpPeerServer.getStreamingAddr();
     LOG.info("Opened streaming server at " + streamingAddr);
     this.threadGroup = new ThreadGroup("dataXceiverServer");
-    xserver = new DataXceiverServer(tcpPeerServer, conf, this);
+    xserver = new DataXceiverServer(tcpPeerServer, getConf(), this);
     this.dataXceiverServer = new Daemon(threadGroup, xserver);
     this.threadGroup.setDaemon(true); // auto destroy when empty
 
-    if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,
-              HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT) ||
-        conf.getBoolean(HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
-              HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
+    if (getConf().getBoolean(
+        HdfsClientConfigKeys.Read.ShortCircuit.KEY,
+        HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT) ||
+        getConf().getBoolean(
+            HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+            HdfsClientConfigKeys
+              .DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
       DomainPeerServer domainPeerServer =
-                getDomainPeerServer(conf, streamingAddr.getPort());
+                getDomainPeerServer(getConf(), streamingAddr.getPort());
       if (domainPeerServer != null) {
         this.localDataXceiverServer = new Daemon(threadGroup,
-            new DataXceiverServer(domainPeerServer, conf, this));
+            new DataXceiverServer(domainPeerServer, getConf(), this));
         LOG.info("Listening on UNIX domain socket: " +
             domainPeerServer.getBindPath());
       }
     }
-    this.shortCircuitRegistry = new ShortCircuitRegistry(conf);
+    this.shortCircuitRegistry = new ShortCircuitRegistry(getConf());
   }
 
   private static DomainPeerServer getDomainPeerServer(Configuration conf,
@@ -1288,26 +1293,23 @@ public class DataNode extends ReconfigurableBase
   /**
    * This method starts the data node with the specified conf.
    * 
-   * @param conf - the configuration
-   *  if conf's CONFIG_PROPERTY_SIMULATED property is set
-   *  then a simulated storage based data node is created.
+   * If conf's CONFIG_PROPERTY_SIMULATED property is set
+   * then a simulated storage based data node is created.
    * 
    * @param dataDirs - only for a non-simulated storage data node
    * @throws IOException
    */
-  void startDataNode(Configuration conf, 
-                     List<StorageLocation> dataDirs,
+  void startDataNode(List<StorageLocation> dataDirectories,
                      SecureResources resources
                      ) throws IOException {
 
     // settings global for all BPs in the Data Node
     this.secureResources = resources;
     synchronized (this) {
-      this.dataDirs = dataDirs;
+      this.dataDirs = dataDirectories;
     }
-    this.conf = conf;
-    this.dnConf = new DNConf(conf);
-    checkSecureConfig(dnConf, conf, resources);
+    this.dnConf = new DNConf(this);
+    checkSecureConfig(dnConf, getConf(), resources);
 
     if (dnConf.maxLockedMemory > 0) {
       if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) {
@@ -1347,10 +1349,10 @@ public class DataNode extends ReconfigurableBase
     
     // global DN settings
     registerMXBean();
-    initDataXceiver(conf);
-    startInfoServer(conf);
+    initDataXceiver();
+    startInfoServer();
     pauseMonitor = new JvmPauseMonitor();
-    pauseMonitor.init(conf);
+    pauseMonitor.init(getConf());
     pauseMonitor.start();
   
     // BlockPoolTokenSecretManager is required to create ipc server.
@@ -1360,24 +1362,24 @@ public class DataNode extends ReconfigurableBase
     dnUserName = UserGroupInformation.getCurrentUser().getShortUserName();
     LOG.info("dnUserName = " + dnUserName);
     LOG.info("supergroup = " + supergroup);
-    initIpcServer(conf);
+    initIpcServer();
 
-    metrics = DataNodeMetrics.create(conf, getDisplayName());
+    metrics = DataNodeMetrics.create(getConf(), getDisplayName());
     metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
 
-    ecWorker = new ErasureCodingWorker(conf, this);
+    ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
 
     blockPoolManager = new BlockPoolManager(this);
-    blockPoolManager.refreshNamenodes(conf);
+    blockPoolManager.refreshNamenodes(getConf());
 
     // Create the ReadaheadPool from the DataNode context so we can
     // exit without having to explicitly shutdown its thread pool.
     readaheadPool = ReadaheadPool.getInstance();
-    saslClient = new SaslDataTransferClient(dnConf.conf, 
+    saslClient = new SaslDataTransferClient(dnConf.getConf(),
         dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
     saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
-    startMetricsLogger(conf);
+    startMetricsLogger();
   }
 
   /**
@@ -1592,10 +1594,10 @@ public class DataNode extends ReconfigurableBase
     // failures.
     checkDiskError();
 
-    data.addBlockPool(nsInfo.getBlockPoolID(), conf);
+    data.addBlockPool(nsInfo.getBlockPoolID(), getConf());
     blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
-    initDirectoryScanner(conf);
-    initDiskBalancer(data, conf);
+    initDirectoryScanner(getConf());
+    initDiskBalancer(data, getConf());
   }
 
   List<BPOfferService> getAllBpOs() {
@@ -1616,10 +1618,10 @@ public class DataNode extends ReconfigurableBase
    */
   private void initStorage(final NamespaceInfo nsInfo) throws IOException {
     final FsDatasetSpi.Factory<? extends FsDatasetSpi<?>> factory
-        = FsDatasetSpi.Factory.getFactory(conf);
+        = FsDatasetSpi.Factory.getFactory(getConf());
     
     if (!factory.isSimulated()) {
-      final StartupOption startOpt = getStartupOption(conf);
+      final StartupOption startOpt = getStartupOption(getConf());
       if (startOpt == null) {
         throw new IOException("Startup option not set.");
       }
@@ -1639,7 +1641,7 @@ public class DataNode extends ReconfigurableBase
 
     synchronized(this)  {
       if (data == null) {
-        data = factory.newInstance(this, storage, conf);
+        data = factory.newInstance(this, storage, getConf());
       }
     }
   }
@@ -1720,7 +1722,7 @@ public class DataNode extends ReconfigurableBase
    */
   DatanodeProtocolClientSideTranslatorPB connectToNN(
       InetSocketAddress nnAddr) throws IOException {
-    return new DatanodeProtocolClientSideTranslatorPB(nnAddr, conf);
+    return new DatanodeProtocolClientSideTranslatorPB(nnAddr, getConf());
   }
 
   /**
@@ -1733,7 +1735,7 @@ public class DataNode extends ReconfigurableBase
   DatanodeLifelineProtocolClientSideTranslatorPB connectToLifelineNN(
       InetSocketAddress lifelineNnAddr) throws IOException {
     return new DatanodeLifelineProtocolClientSideTranslatorPB(lifelineNnAddr,
-        conf);
+        getConf());
   }
 
   public static InterDatanodeProtocol createInterDataNodeProtocolProxy(
@@ -2388,7 +2390,7 @@ public class DataNode extends ReconfigurableBase
         unbufIn = saslStreams.in;
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            DFSUtilClient.getSmallBufferSize(conf)));
+            DFSUtilClient.getSmallBufferSize(getConf())));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);
@@ -2508,7 +2510,7 @@ public class DataNode extends ReconfigurableBase
     }
     ipcServer.setTracer(tracer);
     ipcServer.start();
-    startPlugins(conf);
+    startPlugins(getConf());
   }
 
   /**
@@ -3051,8 +3053,8 @@ public class DataNode extends ReconfigurableBase
   @Override // ClientDatanodeProtocol
   public void refreshNamenodes() throws IOException {
     checkSuperuserPrivilege();
-    conf = new Configuration();
-    refreshNamenodes(conf);
+    setConf(new Configuration());
+    refreshNamenodes(getConf());
   }
   
   @Override // ClientDatanodeProtocol
@@ -3327,8 +3329,8 @@ public class DataNode extends ReconfigurableBase
                            Token<BlockTokenIdentifier> blockToken)
       throws IOException {
 
-    return DFSUtilClient.connectToDN(datanodeID, timeout, conf, saslClient,
-        NetUtils.getDefaultSocketFactory(getConf()), false,
+    return DFSUtilClient.connectToDN(datanodeID, timeout, getConf(),
+        saslClient, NetUtils.getDefaultSocketFactory(getConf()), false,
         getDataEncryptionKeyFactoryForBlock(block), blockToken);
   }
 
@@ -3341,7 +3343,7 @@ public class DataNode extends ReconfigurableBase
     final int numOobTypes = oobEnd - oobStart + 1;
     oobTimeouts = new long[numOobTypes];
 
-    final String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
+    final String[] ele = getConf().get(DFS_DATANODE_OOB_TIMEOUT_KEY,
         DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
     for (int i = 0; i < numOobTypes; i++) {
       oobTimeouts[i] = (i < ele.length) ? Long.parseLong(ele[i]) : 0;
@@ -3367,10 +3369,9 @@ public class DataNode extends ReconfigurableBase
    * Start a timer to periodically write DataNode metrics to the log file. This
    * behavior can be disabled by configuration.
    *
-   * @param metricConf
    */
-  protected void startMetricsLogger(Configuration metricConf) {
-    long metricsLoggerPeriodSec = metricConf.getInt(
+  protected void startMetricsLogger() {
+    long metricsLoggerPeriodSec = getConf().getInt(
         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT);
 
@@ -3474,4 +3475,4 @@ public class DataNode extends ReconfigurableBase
   void setBlockScanner(BlockScanner blockScanner) {
     this.blockScanner = blockScanner;
   }
-}
+}

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

@@ -56,6 +56,6 @@ public class DataNodeFaultInjector {
 
   public void failMirrorConnection() throws IOException { }
 
-  public void failPipeline(ReplicaInPipelineInterface replicaInfo,
+  public void failPipeline(ReplicaInPipeline replicaInfo,
       String mirrorAddr) throws IOException { }
 }

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

@@ -204,9 +204,9 @@ public class DataStorage extends Storage {
    * @return trash directory if rolling upgrade is in progress, null
    *         otherwise.
    */
-  public String getTrashDirectoryForBlockFile(String bpid, File blockFile) {
+  public String getTrashDirectoryForReplica(String bpid, ReplicaInfo info) {
     if (trashEnabledBpids.contains(bpid)) {
-      return getBPStorage(bpid).getTrashDirectory(blockFile);
+      return getBPStorage(bpid).getTrashDirectory(info);
     }
     return null;
   }

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

@@ -597,14 +597,14 @@ public class DirectoryScanner implements Runnable {
         diffs.put(bpid, diffRecord);
         
         statsRecord.totalBlocks = blockpoolReport.length;
-        List<FinalizedReplica> bl = dataset.getFinalizedBlocks(bpid);
-        FinalizedReplica[] memReport = bl.toArray(new FinalizedReplica[bl.size()]);
+        List<ReplicaInfo> bl = dataset.getFinalizedBlocks(bpid);
+        ReplicaInfo[] memReport = bl.toArray(new ReplicaInfo[bl.size()]);
         Arrays.sort(memReport); // Sort based on blockId
   
         int d = 0; // index for blockpoolReport
         int m = 0; // index for memReprot
         while (m < memReport.length && d < blockpoolReport.length) {
-          FinalizedReplica memBlock = memReport[m];
+          ReplicaInfo memBlock = memReport[m];
           ScanInfo info = blockpoolReport[d];
           if (info.getBlockId() < memBlock.getBlockId()) {
             if (!dataset.isDeletingBlock(bpid, info.getBlockId())) {
@@ -633,7 +633,7 @@ public class DirectoryScanner implements Runnable {
             // or block file length is different than expected
             statsRecord.mismatchBlocks++;
             addDifference(diffRecord, statsRecord, info);
-          } else if (info.getBlockFile().compareTo(memBlock.getBlockFile()) != 0) {
+          } else if (memBlock.compareWith(info) != 0) {
             // volumeMap record and on-disk files don't match.
             statsRecord.duplicateBlocks++;
             addDifference(diffRecord, statsRecord, info);
@@ -652,7 +652,7 @@ public class DirectoryScanner implements Runnable {
           }
         }
         while (m < memReport.length) {
-          FinalizedReplica current = memReport[m++];
+          ReplicaInfo current = memReport[m++];
           addDifference(diffRecord, statsRecord,
                         current.getBlockId(), current.getVolume());
         }

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

@@ -22,11 +22,12 @@ import java.io.File;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 
 /**
  * This class describes a replica that has been finalized.
  */
-public class FinalizedReplica extends ReplicaInfo {
+public class FinalizedReplica extends LocalReplica {
 
   /**
    * Constructor
@@ -88,4 +89,28 @@ public class FinalizedReplica extends ReplicaInfo {
   public String toString() {
     return super.toString();
   }
+
+  @Override
+  public ReplicaInfo getOriginalReplica() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getOriginalReplica");
+  }
+
+  @Override
+  public long getRecoveryID() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getRecoveryID");
+  }
+
+  @Override
+  public void setRecoveryID(long recoveryId) {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support setRecoveryID");
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support createInfo");
+  }
 }

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

@@ -0,0 +1,479 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ScanInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.util.DataChecksum;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is used for all replicas which are on local storage media
+ * and hence, are backed by files.
+ */
+abstract public class LocalReplica extends ReplicaInfo {
+
+  /**
+   * Base directory containing numerically-identified sub directories and
+   * possibly blocks.
+   */
+  private File baseDir;
+
+  /**
+   * Whether or not this replica's parent directory includes subdirs, in which
+   * case we can generate them based on the replica's block ID
+   */
+  private boolean hasSubdirs;
+
+  private static final Map<String, File> internedBaseDirs = new HashMap<String, File>();
+
+  static final Log LOG = LogFactory.getLog(LocalReplica.class);
+  private final static boolean IS_NATIVE_IO_AVAIL;
+  static {
+    IS_NATIVE_IO_AVAIL = NativeIO.isAvailable();
+    if (Path.WINDOWS && !IS_NATIVE_IO_AVAIL) {
+      LOG.warn("Data node cannot fully support concurrent reading"
+          + " and writing without native code extensions on Windows.");
+    }
+  }
+
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  LocalReplica(Block block, FsVolumeSpi vol, File dir) {
+    this(block.getBlockId(), block.getNumBytes(),
+        block.getGenerationStamp(), vol, dir);
+  }
+
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  LocalReplica(long blockId, long len, long genStamp,
+      FsVolumeSpi vol, File dir) {
+    super(vol, blockId, len, genStamp);
+    setDirInternal(dir);
+  }
+
+  /**
+   * Copy constructor.
+   * @param from the source replica
+   */
+  LocalReplica(LocalReplica from) {
+    this(from, from.getVolume(), from.getDir());
+  }
+
+  /**
+   * Get the full path of this replica's data file.
+   * @return the full path of this replica's data file
+   */
+  @VisibleForTesting
+  public File getBlockFile() {
+    return new File(getDir(), getBlockName());
+  }
+
+  /**
+   * Get the full path of this replica's meta file.
+   * @return the full path of this replica's meta file
+   */
+  @VisibleForTesting
+  public File getMetaFile() {
+    return new File(getDir(),
+        DatanodeUtil.getMetaName(getBlockName(), getGenerationStamp()));
+  }
+
+  /**
+   * Return the parent directory path where this replica is located.
+   * @return the parent directory path where this replica is located
+   */
+  protected File getDir() {
+    return hasSubdirs ? DatanodeUtil.idToBlockDir(baseDir,
+        getBlockId()) : baseDir;
+  }
+
+  /**
+   * Set the parent directory where this replica is located.
+   * @param dir the parent directory where the replica is located
+   */
+  private void setDirInternal(File dir) {
+    if (dir == null) {
+      baseDir = null;
+      return;
+    }
+
+    ReplicaDirInfo dirInfo = parseBaseDir(dir);
+    this.hasSubdirs = dirInfo.hasSubidrs;
+
+    synchronized (internedBaseDirs) {
+      if (!internedBaseDirs.containsKey(dirInfo.baseDirPath)) {
+        // Create a new String path of this file and make a brand new File object
+        // to guarantee we drop the reference to the underlying char[] storage.
+        File baseDir = new File(dirInfo.baseDirPath);
+        internedBaseDirs.put(dirInfo.baseDirPath, baseDir);
+      }
+      this.baseDir = internedBaseDirs.get(dirInfo.baseDirPath);
+    }
+  }
+
+  @VisibleForTesting
+  public static class ReplicaDirInfo {
+    public String baseDirPath;
+    public boolean hasSubidrs;
+
+    public ReplicaDirInfo (String baseDirPath, boolean hasSubidrs) {
+      this.baseDirPath = baseDirPath;
+      this.hasSubidrs = hasSubidrs;
+    }
+  }
+
+  @VisibleForTesting
+  public static ReplicaDirInfo parseBaseDir(File dir) {
+
+    File currentDir = dir;
+    boolean hasSubdirs = false;
+    while (currentDir.getName().startsWith(DataStorage.BLOCK_SUBDIR_PREFIX)) {
+      hasSubdirs = true;
+      currentDir = currentDir.getParentFile();
+    }
+
+    return new ReplicaDirInfo(currentDir.getAbsolutePath(), hasSubdirs);
+  }
+
+  /**
+   * Copy specified file into a temporary file. Then rename the
+   * temporary file to the original name. This will cause any
+   * hardlinks to the original file to be removed. The temporary
+   * files are created in the same directory. The temporary files will
+   * be recovered (especially on Windows) on datanode restart.
+   */
+  private void breakHardlinks(File file, Block b) throws IOException {
+    File tmpFile = DatanodeUtil.createTmpFile(b, DatanodeUtil.getUnlinkTmpFile(file));
+    try (FileInputStream in = new FileInputStream(file)) {
+      try (FileOutputStream out = new FileOutputStream(tmpFile)){
+        IOUtils.copyBytes(in, out, 16 * 1024);
+      }
+      if (file.length() != tmpFile.length()) {
+        throw new IOException("Copy of file " + file + " size " + file.length()+
+                              " into file " + tmpFile +
+                              " resulted in a size of " + tmpFile.length());
+      }
+      FileUtil.replaceFile(tmpFile, file);
+    } catch (IOException e) {
+      boolean done = tmpFile.delete();
+      if (!done) {
+        DataNode.LOG.info("detachFile failed to delete temporary file " +
+                          tmpFile);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * This function "breaks hardlinks" to the current replica file.
+   *
+   * When doing a DataNode upgrade, we create a bunch of hardlinks to each block
+   * file.  This cleverly ensures that both the old and the new storage
+   * directories can contain the same block file, without using additional space
+   * for the data.
+   *
+   * However, when we want to append to the replica file, we need to "break" the
+   * hardlink to ensure that the old snapshot continues to contain the old data
+   * length.  If we failed to do that, we could roll back to the previous/
+   * directory during a downgrade, and find that the block contents were longer
+   * than they were at the time of upgrade.
+   *
+   * @return true only if data was copied.
+   * @throws IOException
+   */
+  public boolean breakHardLinksIfNeeded() throws IOException {
+    File file = getBlockFile();
+    if (file == null || getVolume() == null) {
+      throw new IOException("detachBlock:Block not found. " + this);
+    }
+    File meta = getMetaFile();
+
+    int linkCount = HardLink.getLinkCount(file);
+    if (linkCount > 1) {
+      DataNode.LOG.info("Breaking hardlink for " + linkCount + "x-linked " +
+          "block " + this);
+      breakHardlinks(file, this);
+    }
+    if (HardLink.getLinkCount(meta) > 1) {
+      breakHardlinks(meta, this);
+    }
+    return true;
+  }
+
+  @Override
+  public URI getBlockURI() {
+    return getBlockFile().toURI();
+  }
+
+  @Override
+  public InputStream getDataInputStream(long seekOffset) throws IOException {
+
+    File blockFile = getBlockFile();
+    if (IS_NATIVE_IO_AVAIL) {
+      return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
+    } else {
+      try {
+        return FsDatasetUtil.openAndSeek(blockFile, seekOffset);
+      } catch (FileNotFoundException fnfe) {
+        throw new IOException("Block " + this + " is not valid. " +
+            "Expected block file at " + blockFile + " does not exist.");
+      }
+    }
+  }
+
+  @Override
+  public OutputStream getDataOutputStream(boolean append) throws IOException {
+    return new FileOutputStream(getBlockFile(), append);
+  }
+
+  @Override
+  public boolean blockDataExists() {
+    return getBlockFile().exists();
+  }
+
+  @Override
+  public boolean deleteBlockData() {
+    return getBlockFile().delete();
+  }
+
+  @Override
+  public long getBlockDataLength() {
+    return getBlockFile().length();
+  }
+
+  @Override
+  public URI getMetadataURI() {
+    return getMetaFile().toURI();
+  }
+
+  @Override
+  public LengthInputStream getMetadataInputStream(long offset)
+      throws IOException {
+    File meta = getMetaFile();
+    return new LengthInputStream(
+        FsDatasetUtil.openAndSeek(meta, offset), meta.length());
+  }
+
+  @Override
+  public OutputStream getMetadataOutputStream(boolean append)
+      throws IOException {
+    return new FileOutputStream(getMetaFile(), append);
+  }
+
+  @Override
+  public boolean metadataExists() {
+    return getMetaFile().exists();
+  }
+
+  @Override
+  public boolean deleteMetadata() {
+    return getMetaFile().delete();
+  }
+
+  @Override
+  public long getMetadataLength() {
+    return getMetaFile().length();
+  }
+
+  @Override
+  public boolean renameMeta(URI destURI) throws IOException {
+    return renameFile(getMetaFile(), new File(destURI));
+  }
+
+  @Override
+  public boolean renameData(URI destURI) throws IOException {
+    return renameFile(getBlockFile(), new File(destURI));
+  }
+
+  private boolean renameFile(File srcfile, File destfile) throws IOException {
+    try {
+      NativeIO.renameTo(srcfile, destfile);
+      return true;
+    } catch (IOException e) {
+      throw new IOException("Failed to move block file for " + this
+          + " from " + srcfile + " to " + destfile.getAbsolutePath(), e);
+    }
+  }
+
+  @Override
+  public void updateWithReplica(StorageLocation replicaLocation) {
+    // for local replicas, the replica location is assumed to be a file.
+    File diskFile = replicaLocation.getFile();
+    if (null == diskFile) {
+      setDirInternal(null);
+    } else {
+      setDirInternal(diskFile.getParentFile());
+    }
+  }
+
+  @Override
+  public boolean getPinning(LocalFileSystem localFS) throws IOException {
+    FileStatus fss =
+        localFS.getFileStatus(new Path(getBlockFile().getAbsolutePath()));
+    return fss.getPermission().getStickyBit();
+  }
+
+  @Override
+  public void setPinning(LocalFileSystem localFS) throws IOException {
+    File f = getBlockFile();
+    Path p = new Path(f.getAbsolutePath());
+
+    FsPermission oldPermission = localFS.getFileStatus(
+        new Path(f.getAbsolutePath())).getPermission();
+    //sticky bit is used for pinning purpose
+    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
+        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
+    localFS.setPermission(p, permission);
+  }
+
+  @Override
+  public void bumpReplicaGS(long newGS) throws IOException {
+    long oldGS = getGenerationStamp();
+    File oldmeta = getMetaFile();
+    setGenerationStamp(newGS);
+    File newmeta = getMetaFile();
+
+    // rename meta file to new GS
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + oldmeta + " to " + newmeta);
+    }
+    try {
+      // calling renameMeta on the ReplicaInfo doesn't work here
+      NativeIO.renameTo(oldmeta, newmeta);
+    } catch (IOException e) {
+      setGenerationStamp(oldGS); // restore old GS
+      throw new IOException("Block " + this + " reopen failed. " +
+                            " Unable to move meta file  " + oldmeta +
+                            " to " + newmeta, e);
+    }
+  }
+
+  @Override
+  public void truncateBlock(long newLength) throws IOException {
+    truncateBlock(getBlockFile(), getMetaFile(), getNumBytes(), newLength);
+  }
+
+  @Override
+  public int compareWith(ScanInfo info) {
+    return info.getBlockFile().compareTo(getBlockFile());
+  }
+
+  static public void truncateBlock(File blockFile, File metaFile,
+      long oldlen, long newlen) throws IOException {
+    LOG.info("truncateBlock: blockFile=" + blockFile
+        + ", metaFile=" + metaFile
+        + ", oldlen=" + oldlen
+        + ", newlen=" + newlen);
+
+    if (newlen == oldlen) {
+      return;
+    }
+    if (newlen > oldlen) {
+      throw new IOException("Cannot truncate block to from oldlen (=" + oldlen
+          + ") to newlen (=" + newlen + ")");
+    }
+
+    DataChecksum dcs = BlockMetadataHeader.readHeader(metaFile).getChecksum();
+    int checksumsize = dcs.getChecksumSize();
+    int bpc = dcs.getBytesPerChecksum();
+    long n = (newlen - 1)/bpc + 1;
+    long newmetalen = BlockMetadataHeader.getHeaderSize() + n*checksumsize;
+    long lastchunkoffset = (n - 1)*bpc;
+    int lastchunksize = (int)(newlen - lastchunkoffset);
+    byte[] b = new byte[Math.max(lastchunksize, checksumsize)];
+
+    RandomAccessFile blockRAF = new RandomAccessFile(blockFile, "rw");
+    try {
+      //truncate blockFile
+      blockRAF.setLength(newlen);
+
+      //read last chunk
+      blockRAF.seek(lastchunkoffset);
+      blockRAF.readFully(b, 0, lastchunksize);
+    } finally {
+      blockRAF.close();
+    }
+
+    //compute checksum
+    dcs.update(b, 0, lastchunksize);
+    dcs.writeValue(b, 0, false);
+
+    //update metaFile
+    RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw");
+    try {
+      metaRAF.setLength(newmetalen);
+      metaRAF.seek(newmetalen - checksumsize);
+      metaRAF.write(b, 0, checksumsize);
+    } finally {
+      metaRAF.close();
+    }
+  }
+
+  @Override
+  public void copyMetadata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    Storage.nativeCopyFileUnbuffered(getMetaFile(),
+        new File(destination), true);
+  }
+
+  @Override
+  public void copyBlockdata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    Storage.nativeCopyFileUnbuffered(getBlockFile(),
+        new File(destination), true);
+  }
+
+}

+ 417 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java

@@ -0,0 +1,417 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * This class defines a replica in a pipeline, which
+ * includes a persistent replica being written to by a dfs client or
+ * a temporary replica being replicated by a source datanode or
+ * being copied for the balancing purpose.
+ *
+ * The base class implements a temporary replica
+ */
+public class LocalReplicaInPipeline extends LocalReplica
+                        implements ReplicaInPipeline {
+  private long bytesAcked;
+  private long bytesOnDisk;
+  private byte[] lastChecksum;
+  private AtomicReference<Thread> writer = new AtomicReference<Thread>();
+
+  /**
+   * Bytes reserved for this replica on the containing volume.
+   * Based off difference between the estimated maximum block length and
+   * the bytes already written to this block.
+   */
+  private long bytesReserved;
+  private final long originalBytesReserved;
+
+  /**
+   * Constructor for a zero length replica.
+   * @param blockId block id
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param bytesToReserve disk space to reserve for this replica, based on
+   *                       the estimated maximum block length.
+   */
+  public LocalReplicaInPipeline(long blockId, long genStamp,
+        FsVolumeSpi vol, File dir, long bytesToReserve) {
+    this(blockId, 0L, genStamp, vol, dir, Thread.currentThread(),
+        bytesToReserve);
+  }
+
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param writer a thread that is writing to this replica
+   */
+  LocalReplicaInPipeline(Block block,
+      FsVolumeSpi vol, File dir, Thread writer) {
+    this(block.getBlockId(), block.getNumBytes(), block.getGenerationStamp(),
+        vol, dir, writer, 0L);
+  }
+
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param writer a thread that is writing to this replica
+   * @param bytesToReserve disk space to reserve for this replica, based on
+   *                       the estimated maximum block length.
+   */
+  LocalReplicaInPipeline(long blockId, long len, long genStamp,
+      FsVolumeSpi vol, File dir, Thread writer, long bytesToReserve) {
+    super(blockId, len, genStamp, vol, dir);
+    this.bytesAcked = len;
+    this.bytesOnDisk = len;
+    this.writer.set(writer);
+    this.bytesReserved = bytesToReserve;
+    this.originalBytesReserved = bytesToReserve;
+  }
+
+  /**
+   * Copy constructor.
+   * @param from where to copy from
+   */
+  public LocalReplicaInPipeline(LocalReplicaInPipeline from) {
+    super(from);
+    this.bytesAcked = from.getBytesAcked();
+    this.bytesOnDisk = from.getBytesOnDisk();
+    this.writer.set(from.writer.get());
+    this.bytesReserved = from.bytesReserved;
+    this.originalBytesReserved = from.originalBytesReserved;
+  }
+
+  @Override
+  public long getVisibleLength() {
+    return -1;
+  }
+
+  @Override  //ReplicaInfo
+  public ReplicaState getState() {
+    return ReplicaState.TEMPORARY;
+  }
+
+  @Override // ReplicaInPipeline
+  public long getBytesAcked() {
+    return bytesAcked;
+  }
+
+  @Override // ReplicaInPipeline
+  public void setBytesAcked(long bytesAcked) {
+    long newBytesAcked = bytesAcked - this.bytesAcked;
+    this.bytesAcked = bytesAcked;
+
+    // Once bytes are ACK'ed we can release equivalent space from the
+    // volume's reservedForRbw count. We could have released it as soon
+    // as the write-to-disk completed but that would be inefficient.
+    getVolume().releaseReservedSpace(newBytesAcked);
+    bytesReserved -= newBytesAcked;
+  }
+
+  @Override // ReplicaInPipeline
+  public long getBytesOnDisk() {
+    return bytesOnDisk;
+  }
+
+  @Override
+  public long getBytesReserved() {
+    return bytesReserved;
+  }
+
+  @Override
+  public long getOriginalBytesReserved() {
+    return originalBytesReserved;
+  }
+
+  @Override // ReplicaInPipeline
+  public void releaseAllBytesReserved() {
+    getVolume().releaseReservedSpace(bytesReserved);
+    getVolume().releaseLockedMemory(bytesReserved);
+    bytesReserved = 0;
+  }
+
+  @Override // ReplicaInPipeline
+  public synchronized void setLastChecksumAndDataLen(long dataLength,
+      byte[] checksum) {
+    this.bytesOnDisk = dataLength;
+    this.lastChecksum = checksum;
+  }
+
+  @Override // ReplicaInPipeline
+  public synchronized ChunkChecksum getLastChecksumAndDataLen() {
+    return new ChunkChecksum(getBytesOnDisk(), lastChecksum);
+  }
+
+  @Override // ReplicaInPipeline
+  public void setWriter(Thread writer) {
+    this.writer.set(writer);
+  }
+
+  @Override
+  public void interruptThread() {
+    Thread thread = writer.get();
+    if (thread != null && thread != Thread.currentThread()
+        && thread.isAlive()) {
+      thread.interrupt();
+    }
+  }
+
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  /**
+   * Attempt to set the writer to a new value.
+   */
+  @Override // ReplicaInPipeline
+  public boolean attemptToSetWriter(Thread prevWriter, Thread newWriter) {
+    return writer.compareAndSet(prevWriter, newWriter);
+  }
+
+  /**
+   * Interrupt the writing thread and wait until it dies.
+   * @throws IOException the waiting is interrupted
+   */
+  @Override // ReplicaInPipeline
+  public void stopWriter(long xceiverStopTimeout) throws IOException {
+    while (true) {
+      Thread thread = writer.get();
+      if ((thread == null) || (thread == Thread.currentThread()) ||
+          (!thread.isAlive())) {
+        if (writer.compareAndSet(thread, null)) {
+          return; // Done
+        }
+        // The writer changed.  Go back to the start of the loop and attempt to
+        // stop the new writer.
+        continue;
+      }
+      thread.interrupt();
+      try {
+        thread.join(xceiverStopTimeout);
+        if (thread.isAlive()) {
+          // Our thread join timed out.
+          final String msg = "Join on writer thread " + thread + " timed out";
+          DataNode.LOG.warn(msg + "\n" + StringUtils.getStackTrace(thread));
+          throw new IOException(msg);
+        }
+      } catch (InterruptedException e) {
+        throw new IOException("Waiting for writer thread is interrupted.");
+      }
+    }
+  }
+
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override // ReplicaInPipeline
+  public ReplicaOutputStreams createStreams(boolean isCreate,
+      DataChecksum requestedChecksum) throws IOException {
+    File blockFile = getBlockFile();
+    File metaFile = getMetaFile();
+    if (DataNode.LOG.isDebugEnabled()) {
+      DataNode.LOG.debug("writeTo blockfile is " + blockFile +
+                         " of size " + blockFile.length());
+      DataNode.LOG.debug("writeTo metafile is " + metaFile +
+                         " of size " + metaFile.length());
+    }
+    long blockDiskSize = 0L;
+    long crcDiskSize = 0L;
+
+    // the checksum that should actually be used -- this
+    // may differ from requestedChecksum for appends.
+    final DataChecksum checksum;
+
+    RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw");
+
+    if (!isCreate) {
+      // For append or recovery, we must enforce the existing checksum.
+      // Also, verify that the file has correct lengths, etc.
+      boolean checkedMeta = false;
+      try {
+        BlockMetadataHeader header = BlockMetadataHeader.readHeader(metaRAF);
+        checksum = header.getChecksum();
+
+        if (checksum.getBytesPerChecksum() !=
+            requestedChecksum.getBytesPerChecksum()) {
+          throw new IOException("Client requested checksum " +
+              requestedChecksum + " when appending to an existing block " +
+              "with different chunk size: " + checksum);
+        }
+
+        int bytesPerChunk = checksum.getBytesPerChecksum();
+        int checksumSize = checksum.getChecksumSize();
+
+        blockDiskSize = bytesOnDisk;
+        crcDiskSize = BlockMetadataHeader.getHeaderSize() +
+          (blockDiskSize+bytesPerChunk-1)/bytesPerChunk*checksumSize;
+        if (blockDiskSize > 0 &&
+            (blockDiskSize > blockFile.length() ||
+               crcDiskSize>metaFile.length())) {
+          throw new IOException("Corrupted block: " + this);
+        }
+        checkedMeta = true;
+      } finally {
+        if (!checkedMeta) {
+          // clean up in case of exceptions.
+          IOUtils.closeStream(metaRAF);
+        }
+      }
+    } else {
+      // for create, we can use the requested checksum
+      checksum = requestedChecksum;
+    }
+
+    FileOutputStream blockOut = null;
+    FileOutputStream crcOut = null;
+    try {
+      blockOut = new FileOutputStream(
+          new RandomAccessFile(blockFile, "rw").getFD());
+      crcOut = new FileOutputStream(metaRAF.getFD());
+      if (!isCreate) {
+        blockOut.getChannel().position(blockDiskSize);
+        crcOut.getChannel().position(crcDiskSize);
+      }
+      return new ReplicaOutputStreams(blockOut, crcOut, checksum,
+          getVolume().isTransientStorage());
+    } catch (IOException e) {
+      IOUtils.closeStream(blockOut);
+      IOUtils.closeStream(metaRAF);
+      throw e;
+    }
+  }
+
+  @Override
+  public OutputStream createRestartMetaStream() throws IOException {
+    File blockFile = getBlockFile();
+    File restartMeta = new File(blockFile.getParent()  +
+        File.pathSeparator + "." + blockFile.getName() + ".restart");
+    if (restartMeta.exists() && !restartMeta.delete()) {
+      DataNode.LOG.warn("Failed to delete restart meta file: " +
+          restartMeta.getPath());
+    }
+    return new FileOutputStream(restartMeta);
+  }
+
+  @Override
+  public String toString() {
+    return super.toString()
+        + "\n  bytesAcked=" + bytesAcked
+        + "\n  bytesOnDisk=" + bytesOnDisk;
+  }
+
+  @Override
+  public ReplicaInfo getOriginalReplica() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getOriginalReplica");
+  }
+
+  @Override
+  public long getRecoveryID() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getRecoveryID");
+  }
+
+  @Override
+  public void setRecoveryID(long recoveryId) {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support setRecoveryID");
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo(){
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support createInfo");
+  }
+
+  public void moveReplicaFrom(ReplicaInfo oldReplicaInfo, File newBlkFile)
+      throws IOException {
+
+    if (!(oldReplicaInfo instanceof LocalReplica)) {
+      throw new IOException("The source replica with blk id "
+          + oldReplicaInfo.getBlockId()
+          + " should be derived from LocalReplica");
+    }
+
+    LocalReplica localReplica = (LocalReplica) oldReplicaInfo;
+
+    File oldmeta = localReplica.getMetaFile();
+    File newmeta = getMetaFile();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + oldmeta + " to " + newmeta);
+    }
+    try {
+      NativeIO.renameTo(oldmeta, newmeta);
+    } catch (IOException e) {
+      throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
+                            " Unable to move meta file  " + oldmeta +
+                            " to rbw dir " + newmeta, e);
+    }
+
+    File blkfile = localReplica.getBlockFile();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + blkfile + " to " + newBlkFile
+          + ", file length=" + blkfile.length());
+    }
+    try {
+      NativeIO.renameTo(blkfile, newBlkFile);
+    } catch (IOException e) {
+      try {
+        NativeIO.renameTo(newmeta, oldmeta);
+      } catch (IOException ex) {
+        LOG.warn("Cannot move meta file " + newmeta +
+            "back to the finalized directory " + oldmeta, ex);
+      }
+      throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
+                              " Unable to move block file " + blkfile +
+                              " to rbw dir " + newBlkFile, e);
+    }
+  }
+
+  @Override // ReplicaInPipeline
+  public ReplicaInfo getReplicaInfo() {
+    return this;
+  }
+}

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java

@@ -27,9 +27,9 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
  * Those are the replicas that
  * are created in a pipeline initiated by a dfs client.
  */
-public class ReplicaBeingWritten extends ReplicaInPipeline {
+public class ReplicaBeingWritten extends LocalReplicaInPipeline {
   /**
-   * Constructor for a zero length replica
+   * Constructor for a zero length replica.
    * @param blockId block id
    * @param genStamp replica generation stamp
    * @param vol volume where replica is located
@@ -37,25 +37,25 @@ public class ReplicaBeingWritten extends ReplicaInPipeline {
    * @param bytesToReserve disk space to reserve for this replica, based on
    *                       the estimated maximum block length.
    */
-  public ReplicaBeingWritten(long blockId, long genStamp, 
+  public ReplicaBeingWritten(long blockId, long genStamp,
         FsVolumeSpi vol, File dir, long bytesToReserve) {
     super(blockId, genStamp, vol, dir, bytesToReserve);
   }
-  
+
   /**
-   * Constructor
+   * Constructor.
    * @param block a block
    * @param vol volume where replica is located
    * @param dir directory path where block and meta files are located
    * @param writer a thread that is writing to this replica
    */
-  public ReplicaBeingWritten(Block block, 
+  public ReplicaBeingWritten(Block block,
       FsVolumeSpi vol, File dir, Thread writer) {
-    super( block, vol, dir, writer);
+    super(block, vol, dir, writer);
   }
 
   /**
-   * Constructor
+   * Constructor.
    * @param blockId block id
    * @param len replica length
    * @param genStamp replica generation stamp

+ 252 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java

@@ -0,0 +1,252 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+
+/**
+ * This class is to be used as a builder for {@link ReplicaInfo} objects.
+ * The state of the replica is used to determine which object is instantiated.
+ */
+public class ReplicaBuilder {
+
+  private ReplicaState state;
+  private long blockId;
+  private long genStamp;
+  private long length;
+  private FsVolumeSpi volume;
+  private File directoryUsed;
+  private long bytesToReserve;
+  private Thread writer;
+  private long recoveryId;
+  private Block block;
+
+  private ReplicaInfo fromReplica;
+
+  public ReplicaBuilder(ReplicaState state) {
+    volume = null;
+    writer = null;
+    block = null;
+    length = -1;
+    this.state = state;
+  }
+
+  public ReplicaBuilder setState(ReplicaState state) {
+    this.state = state;
+    return this;
+  }
+
+  public ReplicaBuilder setBlockId(long blockId) {
+    this.blockId = blockId;
+    return this;
+  }
+
+  public ReplicaBuilder setGenerationStamp(long genStamp) {
+    this.genStamp = genStamp;
+    return this;
+  }
+
+  public ReplicaBuilder setLength(long length) {
+    this.length = length;
+    return this;
+  }
+
+  public ReplicaBuilder setFsVolume(FsVolumeSpi volume) {
+    this.volume = volume;
+    return this;
+  }
+
+  public ReplicaBuilder setDirectoryToUse(File dir) {
+    this.directoryUsed = dir;
+    return this;
+  }
+
+  public ReplicaBuilder setBytesToReserve(long bytesToReserve) {
+    this.bytesToReserve = bytesToReserve;
+    return this;
+  }
+
+  public ReplicaBuilder setWriterThread(Thread writer) {
+    this.writer = writer;
+    return this;
+  }
+
+  public ReplicaBuilder from(ReplicaInfo fromReplica) {
+    this.fromReplica = fromReplica;
+    return this;
+  }
+
+  public ReplicaBuilder setRecoveryId(long recoveryId) {
+    this.recoveryId = recoveryId;
+    return this;
+  }
+
+  public ReplicaBuilder setBlock(Block block) {
+    this.block = block;
+    return this;
+  }
+
+  public LocalReplicaInPipeline buildLocalReplicaInPipeline()
+      throws IllegalArgumentException {
+    LocalReplicaInPipeline info = null;
+    switch(state) {
+    case RBW:
+      info = buildRBW();
+      break;
+    case TEMPORARY:
+      info = buildTemporaryReplica();
+      break;
+    default:
+      throw new IllegalArgumentException("Unknown replica state " + state);
+    }
+    return info;
+  }
+
+  private LocalReplicaInPipeline buildRBW() throws IllegalArgumentException {
+    if (null != fromReplica && fromReplica.getState() == ReplicaState.RBW) {
+      return new ReplicaBeingWritten((ReplicaBeingWritten) fromReplica);
+    } else if (null != fromReplica) {
+      throw new IllegalArgumentException("Incompatible fromReplica "
+          + "state: " + fromReplica.getState());
+    } else {
+      if (null != block) {
+        if (null == writer) {
+          throw new IllegalArgumentException("A valid writer is "
+              + "required for constructing a RBW from block "
+              + block.getBlockId());
+        }
+        return new ReplicaBeingWritten(block, volume, directoryUsed, writer);
+      } else {
+        if (length != -1) {
+          return new ReplicaBeingWritten(blockId, length, genStamp,
+              volume, directoryUsed, writer, bytesToReserve);
+        } else {
+          return new ReplicaBeingWritten(blockId, genStamp, volume,
+              directoryUsed, bytesToReserve);
+        }
+      }
+    }
+  }
+
+  private LocalReplicaInPipeline buildTemporaryReplica()
+      throws IllegalArgumentException {
+    if (null != fromReplica &&
+        fromReplica.getState() == ReplicaState.TEMPORARY) {
+      return new LocalReplicaInPipeline((LocalReplicaInPipeline) fromReplica);
+    } else if (null != fromReplica) {
+      throw new IllegalArgumentException("Incompatible fromReplica "
+          + "state: " + fromReplica.getState());
+    } else {
+      if (null != block) {
+        if (null == writer) {
+          throw new IllegalArgumentException("A valid writer is "
+              + "required for constructing a Replica from block "
+              + block.getBlockId());
+        }
+        return new LocalReplicaInPipeline(block, volume, directoryUsed,
+            writer);
+      } else {
+        if (length != -1) {
+          return new LocalReplicaInPipeline(blockId, length, genStamp,
+              volume, directoryUsed, writer, bytesToReserve);
+        } else {
+          return new LocalReplicaInPipeline(blockId, genStamp, volume,
+              directoryUsed, bytesToReserve);
+        }
+      }
+    }
+  }
+
+  private ReplicaInfo buildFinalizedReplica() throws IllegalArgumentException {
+    if (null != fromReplica &&
+        fromReplica.getState() == ReplicaState.FINALIZED) {
+      return new FinalizedReplica((FinalizedReplica)fromReplica);
+    } else if (null != this.fromReplica) {
+      throw new IllegalArgumentException("Incompatible fromReplica "
+          + "state: " + fromReplica.getState());
+    } else {
+      if (null != block) {
+        return new FinalizedReplica(block, volume, directoryUsed);
+      } else {
+        return new FinalizedReplica(blockId, length, genStamp, volume,
+            directoryUsed);
+      }
+    }
+  }
+
+  private ReplicaInfo buildRWR() throws IllegalArgumentException {
+
+    if (null != fromReplica && fromReplica.getState() == ReplicaState.RWR) {
+      return new ReplicaWaitingToBeRecovered(
+          (ReplicaWaitingToBeRecovered) fromReplica);
+    } else if (null != fromReplica){
+      throw new IllegalArgumentException("Incompatible fromReplica "
+          + "state: " + fromReplica.getState());
+    } else {
+      if (null != block) {
+        return new ReplicaWaitingToBeRecovered(block, volume, directoryUsed);
+      } else {
+        return new ReplicaWaitingToBeRecovered(blockId, length, genStamp,
+            volume, directoryUsed);
+      }
+    }
+  }
+
+  private ReplicaInfo buildRUR() throws IllegalArgumentException {
+    if (null == fromReplica) {
+      throw new IllegalArgumentException(
+          "Missing a valid replica to recover from");
+    }
+    if (null != writer || null != block) {
+      throw new IllegalArgumentException("Invalid state for "
+          + "recovering from replica with blk id "
+          + fromReplica.getBlockId());
+    }
+    if (fromReplica.getState() == ReplicaState.RUR) {
+      return new ReplicaUnderRecovery((ReplicaUnderRecovery) fromReplica);
+    } else {
+      return new ReplicaUnderRecovery(fromReplica, recoveryId);
+    }
+  }
+
+  public ReplicaInfo build() throws IllegalArgumentException {
+    ReplicaInfo info = null;
+    switch(this.state) {
+    case FINALIZED:
+      info = buildFinalizedReplica();
+      break;
+    case RWR:
+      info = buildRWR();
+      break;
+    case RUR:
+      info = buildRUR();
+      break;
+    case RBW:
+    case TEMPORARY:
+      info = buildLocalReplicaInPipeline();
+      break;
+    default:
+      throw new IllegalArgumentException("Unknown replica state " + state);
+    }
+    return info;
+  }
+}

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaHandler.java

@@ -27,11 +27,11 @@ import java.io.IOException;
  * the fs volume where this replica is located.
  */
 public class ReplicaHandler implements Closeable {
-  private final ReplicaInPipelineInterface replica;
+  private final ReplicaInPipeline replica;
   private final FsVolumeReference volumeReference;
 
   public ReplicaHandler(
-      ReplicaInPipelineInterface replica, FsVolumeReference reference) {
+      ReplicaInPipeline replica, FsVolumeReference reference) {
     this.replica = replica;
     this.volumeReference = reference;
   }
@@ -43,7 +43,7 @@ public class ReplicaHandler implements Closeable {
     }
   }
 
-  public ReplicaInPipelineInterface getReplica() {
+  public ReplicaInPipeline getReplica() {
     return replica;
   }
 }

+ 51 - 273
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java

@@ -17,313 +17,91 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.io.RandomAccessFile;
-import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.StringUtils;
 
 /** 
- * This class defines a replica in a pipeline, which
- * includes a persistent replica being written to by a dfs client or
- * a temporary replica being replicated by a source datanode or
- * being copied for the balancing purpose.
- * 
- * The base class implements a temporary replica
+ * This defines the interface of a replica in Pipeline that's being written to
  */
-public class ReplicaInPipeline extends ReplicaInfo
-                        implements ReplicaInPipelineInterface {
-  private long bytesAcked;
-  private long bytesOnDisk;
-  private byte[] lastChecksum;  
-  private AtomicReference<Thread> writer = new AtomicReference<Thread>();
-
+public interface ReplicaInPipeline extends Replica {
   /**
-   * Bytes reserved for this replica on the containing volume.
-   * Based off difference between the estimated maximum block length and
-   * the bytes already written to this block.
+   * Set the number of bytes received
+   * @param bytesReceived number of bytes received
    */
-  private long bytesReserved;
-  private final long originalBytesReserved;
+  void setNumBytes(long bytesReceived);
 
   /**
-   * Constructor for a zero length replica
-   * @param blockId block id
-   * @param genStamp replica generation stamp
-   * @param vol volume where replica is located
-   * @param dir directory path where block and meta files are located
-   * @param bytesToReserve disk space to reserve for this replica, based on
-   *                       the estimated maximum block length.
+   * Get the number of bytes acked
+   * @return the number of bytes acked
    */
-  public ReplicaInPipeline(long blockId, long genStamp, 
-        FsVolumeSpi vol, File dir, long bytesToReserve) {
-    this(blockId, 0L, genStamp, vol, dir, Thread.currentThread(), bytesToReserve);
-  }
+  long getBytesAcked();
 
   /**
-   * Constructor
-   * @param block a block
-   * @param vol volume where replica is located
-   * @param dir directory path where block and meta files are located
-   * @param writer a thread that is writing to this replica
+   * Set the number bytes that have acked
+   * @param bytesAcked number bytes acked
    */
-  ReplicaInPipeline(Block block, 
-      FsVolumeSpi vol, File dir, Thread writer) {
-    this( block.getBlockId(), block.getNumBytes(), block.getGenerationStamp(),
-        vol, dir, writer, 0L);
-  }
+  void setBytesAcked(long bytesAcked);
 
   /**
-   * Constructor
-   * @param blockId block id
-   * @param len replica length
-   * @param genStamp replica generation stamp
-   * @param vol volume where replica is located
-   * @param dir directory path where block and meta files are located
-   * @param writer a thread that is writing to this replica
-   * @param bytesToReserve disk space to reserve for this replica, based on
-   *                       the estimated maximum block length.
+   * Release any disk space reserved for this replica.
    */
-  ReplicaInPipeline(long blockId, long len, long genStamp,
-      FsVolumeSpi vol, File dir, Thread writer, long bytesToReserve) {
-    super( blockId, len, genStamp, vol, dir);
-    this.bytesAcked = len;
-    this.bytesOnDisk = len;
-    this.writer.set(writer);
-    this.bytesReserved = bytesToReserve;
-    this.originalBytesReserved = bytesToReserve;
-  }
+  public void releaseAllBytesReserved();
 
   /**
-   * Copy constructor.
-   * @param from where to copy from
+   * store the checksum for the last chunk along with the data length
+   * @param dataLength number of bytes on disk
+   * @param lastChecksum - checksum bytes for the last chunk
    */
-  public ReplicaInPipeline(ReplicaInPipeline from) {
-    super(from);
-    this.bytesAcked = from.getBytesAcked();
-    this.bytesOnDisk = from.getBytesOnDisk();
-    this.writer.set(from.writer.get());
-    this.bytesReserved = from.bytesReserved;
-    this.originalBytesReserved = from.originalBytesReserved;
-  }
-
-  @Override
-  public long getVisibleLength() {
-    return -1;
-  }
-  
-  @Override  //ReplicaInfo
-  public ReplicaState getState() {
-    return ReplicaState.TEMPORARY;
-  }
+  public void setLastChecksumAndDataLen(long dataLength, byte[] lastChecksum);
   
-  @Override // ReplicaInPipelineInterface
-  public long getBytesAcked() {
-    return bytesAcked;
-  }
+  /**
+   * gets the last chunk checksum and the length of the block corresponding
+   * to that checksum
+   */
+  public ChunkChecksum getLastChecksumAndDataLen();
   
-  @Override // ReplicaInPipelineInterface
-  public void setBytesAcked(long bytesAcked) {
-    long newBytesAcked = bytesAcked - this.bytesAcked;
-    this.bytesAcked = bytesAcked;
+  /**
+   * Create output streams for writing to this replica,
+   * one for block file and one for CRC file
+   *
+   * @param isCreate if it is for creation
+   * @param requestedChecksum the checksum the writer would prefer to use
+   * @return output streams for writing
+   * @throws IOException if any error occurs
+   */
+  public ReplicaOutputStreams createStreams(boolean isCreate,
+      DataChecksum requestedChecksum) throws IOException;
 
-    // Once bytes are ACK'ed we can release equivalent space from the
-    // volume's reservedForRbw count. We could have released it as soon
-    // as the write-to-disk completed but that would be inefficient.
-    getVolume().releaseReservedSpace(newBytesAcked);
-    bytesReserved -= newBytesAcked;
-  }
+  /**
+   * Create an output stream to write restart metadata in case of datanode
+   * shutting down for quick restart.
+   *
+   * @return output stream for writing.
+   * @throws IOException if any error occurs
+   */
+  public OutputStream createRestartMetaStream() throws IOException;
   
-  @Override // ReplicaInPipelineInterface
-  public long getBytesOnDisk() {
-    return bytesOnDisk;
-  }
-
-  @Override
-  public long getBytesReserved() {
-    return bytesReserved;
-  }
+  ReplicaInfo getReplicaInfo();
   
-  @Override
-  public long getOriginalBytesReserved() {
-    return originalBytesReserved;
-  }
-
-  @Override
-  public void releaseAllBytesReserved() {  // ReplicaInPipelineInterface
-    getVolume().releaseReservedSpace(bytesReserved);
-    getVolume().releaseLockedMemory(bytesReserved);
-    bytesReserved = 0;
-  }
-
-  @Override // ReplicaInPipelineInterface
-  public synchronized void setLastChecksumAndDataLen(long dataLength, byte[] lastChecksum) {
-    this.bytesOnDisk = dataLength;
-    this.lastChecksum = lastChecksum;
-  }
+  /**
+   * Set the thread that is writing to this replica
+   * @param writer a thread writing to this replica
+   */
+  void setWriter(Thread writer);
   
-  @Override // ReplicaInPipelineInterface
-  public synchronized ChunkChecksum getLastChecksumAndDataLen() {
-    return new ChunkChecksum(getBytesOnDisk(), lastChecksum);
-  }
-
-  public void interruptThread() {
-    Thread thread = writer.get();
-    if (thread != null && thread != Thread.currentThread() 
-        && thread.isAlive()) {
-      thread.interrupt();
-    }
-  }
-
-  @Override  // Object
-  public boolean equals(Object o) {
-    return super.equals(o);
-  }
+  void interruptThread();
   
   /**
    * Attempt to set the writer to a new value.
    */
-  public boolean attemptToSetWriter(Thread prevWriter, Thread newWriter) {
-    return writer.compareAndSet(prevWriter, newWriter);
-  }
+  boolean attemptToSetWriter(Thread prevWriter, Thread newWriter);
 
   /**
-   * Interrupt the writing thread and wait until it dies
+   * Interrupt the writing thread and wait until it dies.
    * @throws IOException the waiting is interrupted
    */
-  public void stopWriter(long xceiverStopTimeout) throws IOException {
-    while (true) {
-      Thread thread = writer.get();
-      if ((thread == null) || (thread == Thread.currentThread()) ||
-          (!thread.isAlive())) {
-        if (writer.compareAndSet(thread, null) == true) {
-          return; // Done
-        }
-        // The writer changed.  Go back to the start of the loop and attempt to
-        // stop the new writer.
-        continue;
-      }
-      thread.interrupt();
-      try {
-        thread.join(xceiverStopTimeout);
-        if (thread.isAlive()) {
-          // Our thread join timed out.
-          final String msg = "Join on writer thread " + thread + " timed out";
-          DataNode.LOG.warn(msg + "\n" + StringUtils.getStackTrace(thread));
-          throw new IOException(msg);
-        }
-      } catch (InterruptedException e) {
-        throw new IOException("Waiting for writer thread is interrupted.");
-      }
-    }
-  }
-
-  @Override  // Object
-  public int hashCode() {
-    return super.hashCode();
-  }
-  
-  @Override // ReplicaInPipelineInterface
-  public ReplicaOutputStreams createStreams(boolean isCreate, 
-      DataChecksum requestedChecksum) throws IOException {
-    File blockFile = getBlockFile();
-    File metaFile = getMetaFile();
-    if (DataNode.LOG.isDebugEnabled()) {
-      DataNode.LOG.debug("writeTo blockfile is " + blockFile +
-                         " of size " + blockFile.length());
-      DataNode.LOG.debug("writeTo metafile is " + metaFile +
-                         " of size " + metaFile.length());
-    }
-    long blockDiskSize = 0L;
-    long crcDiskSize = 0L;
-    
-    // the checksum that should actually be used -- this
-    // may differ from requestedChecksum for appends.
-    final DataChecksum checksum;
-    
-    RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw");
-    
-    if (!isCreate) {
-      // For append or recovery, we must enforce the existing checksum.
-      // Also, verify that the file has correct lengths, etc.
-      boolean checkedMeta = false;
-      try {
-        BlockMetadataHeader header = BlockMetadataHeader.readHeader(metaRAF);
-        checksum = header.getChecksum();
-        
-        if (checksum.getBytesPerChecksum() !=
-            requestedChecksum.getBytesPerChecksum()) {
-          throw new IOException("Client requested checksum " +
-              requestedChecksum + " when appending to an existing block " +
-              "with different chunk size: " + checksum);
-        }
-        
-        int bytesPerChunk = checksum.getBytesPerChecksum();
-        int checksumSize = checksum.getChecksumSize();
-        
-        blockDiskSize = bytesOnDisk;
-        crcDiskSize = BlockMetadataHeader.getHeaderSize() +
-          (blockDiskSize+bytesPerChunk-1)/bytesPerChunk*checksumSize;
-        if (blockDiskSize>0 && 
-            (blockDiskSize>blockFile.length() || crcDiskSize>metaFile.length())) {
-          throw new IOException("Corrupted block: " + this);
-        }
-        checkedMeta = true;
-      } finally {
-        if (!checkedMeta) {
-          // clean up in case of exceptions.
-          IOUtils.closeStream(metaRAF);
-        }
-      }
-    } else {
-      // for create, we can use the requested checksum
-      checksum = requestedChecksum;
-    }
-    
-    FileOutputStream blockOut = null;
-    FileOutputStream crcOut = null;
-    try {
-      blockOut = new FileOutputStream(
-          new RandomAccessFile( blockFile, "rw" ).getFD() );
-      crcOut = new FileOutputStream(metaRAF.getFD() );
-      if (!isCreate) {
-        blockOut.getChannel().position(blockDiskSize);
-        crcOut.getChannel().position(crcDiskSize);
-      }
-      return new ReplicaOutputStreams(blockOut, crcOut, checksum,
-          getVolume().isTransientStorage());
-    } catch (IOException e) {
-      IOUtils.closeStream(blockOut);
-      IOUtils.closeStream(metaRAF);
-      throw e;
-    }
-  }
-
-  @Override
-  public OutputStream createRestartMetaStream() throws IOException {
-    File blockFile = getBlockFile();
-    File restartMeta = new File(blockFile.getParent()  +
-        File.pathSeparator + "." + blockFile.getName() + ".restart");
-    if (restartMeta.exists() && !restartMeta.delete()) {
-      DataNode.LOG.warn("Failed to delete restart meta file: " +
-          restartMeta.getPath());
-    }
-    return new FileOutputStream(restartMeta);
-  }
-
-  @Override
-  public String toString() {
-    return super.toString()
-        + "\n  bytesAcked=" + bytesAcked
-        + "\n  bytesOnDisk=" + bytesOnDisk;
-  }
+  void stopWriter(long xceiverStopTimeout) throws IOException;
 }

+ 0 - 86
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java

@@ -1,86 +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.hdfs.server.datanode;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.util.DataChecksum;
-
-/** 
- * This defines the interface of a replica in Pipeline that's being written to
- */
-public interface ReplicaInPipelineInterface extends Replica {
-  /**
-   * Set the number of bytes received
-   * @param bytesReceived number of bytes received
-   */
-  void setNumBytes(long bytesReceived);
-  
-  /**
-   * Get the number of bytes acked
-   * @return the number of bytes acked
-   */
-  long getBytesAcked();
-  
-  /**
-   * Set the number bytes that have acked
-   * @param bytesAcked number bytes acked
-   */
-  void setBytesAcked(long bytesAcked);
-  
-  /**
-   * Release any disk space reserved for this replica.
-   */
-  public void releaseAllBytesReserved();
-
-  /**
-   * store the checksum for the last chunk along with the data length
-   * @param dataLength number of bytes on disk
-   * @param lastChecksum - checksum bytes for the last chunk
-   */
-  public void setLastChecksumAndDataLen(long dataLength, byte[] lastChecksum);
-  
-  /**
-   * gets the last chunk checksum and the length of the block corresponding
-   * to that checksum
-   */
-  public ChunkChecksum getLastChecksumAndDataLen();
-  
-  /**
-   * Create output streams for writing to this replica, 
-   * one for block file and one for CRC file
-   * 
-   * @param isCreate if it is for creation
-   * @param requestedChecksum the checksum the writer would prefer to use
-   * @return output streams for writing
-   * @throws IOException if any error occurs
-   */
-  public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum) throws IOException;
-
-  /**
-   * Create an output stream to write restart metadata in case of datanode
-   * shutting down for quick restart.
-   *
-   * @return output stream for writing.
-   * @throws IOException if any error occurs
-   */
-  public OutputStream createRestartMetaStream() throws IOException;
-}

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

@@ -17,23 +17,20 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.util.LightWeightResizableGSet;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * This class is used by datanodes to maintain meta data of its replicas.
  * It provides a general interface for meta information of a replica.
@@ -42,81 +39,26 @@ import com.google.common.annotations.VisibleForTesting;
 abstract public class ReplicaInfo extends Block
     implements Replica, LightWeightResizableGSet.LinkedElement {
 
-  /** For implementing {@link LightWeightResizableGSet.LinkedElement} interface */
+  /** For implementing {@link LightWeightResizableGSet.LinkedElement}. */
   private LightWeightResizableGSet.LinkedElement next;
 
-  /** volume where the replica belongs */
+  /** volume where the replica belongs. */
   private FsVolumeSpi volume;
-  
-  /** directory where block & meta files belong */
-  
-  /**
-   * Base directory containing numerically-identified sub directories and
-   * possibly blocks.
-   */
-  private File baseDir;
-  
-  /**
-   * Whether or not this replica's parent directory includes subdirs, in which
-   * case we can generate them based on the replica's block ID
-   */
-  private boolean hasSubdirs;
-  
-  private static final Map<String, File> internedBaseDirs = new HashMap<String, File>();
 
   /**
-   * Constructor
-   * @param block a block
-   * @param vol volume where replica is located
-   * @param dir directory path where block and meta files are located
-   */
-  ReplicaInfo(Block block, FsVolumeSpi vol, File dir) {
-    this(block.getBlockId(), block.getNumBytes(), 
-        block.getGenerationStamp(), vol, dir);
-  }
-  
-  /**
-   * Constructor
-   * @param blockId block id
-   * @param len replica length
-   * @param genStamp replica generation stamp
-   * @param vol volume where replica is located
-   * @param dir directory path where block and meta files are located
-   */
-  ReplicaInfo(long blockId, long len, long genStamp,
-      FsVolumeSpi vol, File dir) {
+  * Constructor
+  * @param vol volume where replica is located
+  * @param blockId block id
+  * @param len replica length
+  * @param genStamp replica generation stamp
+  */
+  ReplicaInfo(FsVolumeSpi vol, long blockId, long len, long genStamp) {
     super(blockId, len, genStamp);
     this.volume = vol;
-    setDirInternal(dir);
-  }
-
-  /**
-   * Copy constructor.
-   * @param from where to copy from
-   */
-  ReplicaInfo(ReplicaInfo from) {
-    this(from, from.getVolume(), from.getDir());
-  }
-  
-  /**
-   * Get the full path of this replica's data file
-   * @return the full path of this replica's data file
-   */
-  public File getBlockFile() {
-    return new File(getDir(), getBlockName());
   }
   
   /**
-   * Get the full path of this replica's meta file
-   * @return the full path of this replica's meta file
-   */
-  public File getMetaFile() {
-    return new File(getDir(),
-        DatanodeUtil.getMetaName(getBlockName(), getGenerationStamp()));
-  }
-  
-  /**
-   * Get the volume where this replica is located on disk
+   * Get the volume where this replica is located on disk.
    * @return the volume where this replica is located on disk
    */
   public FsVolumeSpi getVolume() {
@@ -124,7 +66,7 @@ abstract public class ReplicaInfo extends Block
   }
   
   /**
-   * Set the volume where this replica is located on disk
+   * Set the volume where this replica is located on disk.
    */
   void setVolume(FsVolumeSpi vol) {
     this.volume = vol;
@@ -137,156 +79,182 @@ abstract public class ReplicaInfo extends Block
   public String getStorageUuid() {
     return volume.getStorageID();
   }
-  
+
   /**
-   * Return the parent directory path where this replica is located
-   * @return the parent directory path where this replica is located
+   * Number of bytes reserved for this replica on disk.
    */
-  File getDir() {
-    return hasSubdirs ? DatanodeUtil.idToBlockDir(baseDir,
-        getBlockId()) : baseDir;
+  public long getBytesReserved() {
+    return 0;
   }
 
   /**
-   * Set the parent directory where this replica is located
-   * @param dir the parent directory where the replica is located
+   * Get the {@code URI} for where the data of this replica is stored.
+   * @return {@code URI} for the location of replica data.
    */
-  public void setDir(File dir) {
-    setDirInternal(dir);
-  }
+  abstract public URI getBlockURI();
 
-  private void setDirInternal(File dir) {
-    if (dir == null) {
-      baseDir = null;
-      return;
-    }
-
-    ReplicaDirInfo dirInfo = parseBaseDir(dir);
-    this.hasSubdirs = dirInfo.hasSubidrs;
-    
-    synchronized (internedBaseDirs) {
-      if (!internedBaseDirs.containsKey(dirInfo.baseDirPath)) {
-        // Create a new String path of this file and make a brand new File object
-        // to guarantee we drop the reference to the underlying char[] storage.
-        File baseDir = new File(dirInfo.baseDirPath);
-        internedBaseDirs.put(dirInfo.baseDirPath, baseDir);
-      }
-      this.baseDir = internedBaseDirs.get(dirInfo.baseDirPath);
-    }
-  }
+  /**
+   * Returns an {@link InputStream} to the replica's data.
+   * @param seekOffset the offset at which the read is started from.
+   * @return the {@link InputStream} to read the replica data.
+   * @throws IOException if an error occurs in opening a stream to the data.
+   */
+  abstract public InputStream getDataInputStream(long seekOffset)
+      throws IOException;
 
-  @VisibleForTesting
-  public static class ReplicaDirInfo {
-    public String baseDirPath;
-    public boolean hasSubidrs;
+  /**
+   * Returns an {@link OutputStream} to the replica's data.
+   * @param append indicates if the block should be opened for append.
+   * @return the {@link OutputStream} to write to the replica.
+   * @throws IOException if an error occurs in creating an {@link OutputStream}.
+   */
+  abstract public OutputStream getDataOutputStream(boolean append)
+      throws IOException;
 
-    public ReplicaDirInfo (String baseDirPath, boolean hasSubidrs) {
-      this.baseDirPath = baseDirPath;
-      this.hasSubidrs = hasSubidrs;
-    }
-  }
-  
-  @VisibleForTesting
-  public static ReplicaDirInfo parseBaseDir(File dir) {
-    
-    File currentDir = dir;
-    boolean hasSubdirs = false;
-    while (currentDir.getName().startsWith(DataStorage.BLOCK_SUBDIR_PREFIX)) {
-      hasSubdirs = true;
-      currentDir = currentDir.getParentFile();
-    }
-    
-    return new ReplicaDirInfo(currentDir.getAbsolutePath(), hasSubdirs);
-  }
+  /**
+   * @return true if the replica's data exists.
+   */
+  abstract public boolean blockDataExists();
 
   /**
-   * Number of bytes reserved for this replica on disk.
+   * Used to deletes the replica's block data.
+   *
+   * @return true if the replica's data is successfully deleted.
    */
-  public long getBytesReserved() {
-    return 0;
-  }
+  abstract public boolean deleteBlockData();
 
   /**
-   * Number of bytes originally reserved for this replica. The actual
-   * reservation is adjusted as data is written to disk.
+   * @return the length of the block on storage.
+   */
+  abstract public long getBlockDataLength();
+
+  /**
+   * Get the {@code URI} for where the metadata of this replica is stored.
    *
-   * @return the number of bytes originally reserved for this replica.
+   * @return {@code URI} for the location of replica metadata.
    */
-  public long getOriginalBytesReserved() {
-    return 0;
-  }
+  abstract public URI getMetadataURI();
 
   /**
-   * Copy specified file into a temporary file. Then rename the
-   * temporary file to the original name. This will cause any
-   * hardlinks to the original file to be removed. The temporary
-   * files are created in the same directory. The temporary files will
-   * be recovered (especially on Windows) on datanode restart.
+   * Returns an {@link InputStream} to the replica's metadata.
+   * @param offset the offset at which the read is started from.
+   * @return the {@link LengthInputStream} to read the replica metadata.
+   * @throws IOException
    */
-  private void breakHardlinks(File file, Block b) throws IOException {
-    File tmpFile = DatanodeUtil.createTmpFile(b, DatanodeUtil.getUnlinkTmpFile(file));
-    try {
-      FileInputStream in = new FileInputStream(file);
-      try {
-        FileOutputStream out = new FileOutputStream(tmpFile);
-        try {
-          IOUtils.copyBytes(in, out, 16 * 1024);
-        } finally {
-          out.close();
-        }
-      } finally {
-        in.close();
-      }
-      if (file.length() != tmpFile.length()) {
-        throw new IOException("Copy of file " + file + " size " + file.length()+
-                              " into file " + tmpFile +
-                              " resulted in a size of " + tmpFile.length());
-      }
-      FileUtil.replaceFile(tmpFile, file);
-    } catch (IOException e) {
-      boolean done = tmpFile.delete();
-      if (!done) {
-        DataNode.LOG.info("detachFile failed to delete temporary file " +
-                          tmpFile);
-      }
-      throw e;
-    }
-  }
+  abstract public LengthInputStream getMetadataInputStream(long offset)
+      throws IOException;
+
+  /**
+   * Returns an {@link OutputStream} to the replica's metadata.
+   * @param append indicates if the block metadata should be opened for append.
+   * @return the {@link OutputStream} to write to the replica's metadata.
+   * @throws IOException if an error occurs in creating an {@link OutputStream}.
+   */
+  abstract public OutputStream getMetadataOutputStream(boolean append)
+      throws IOException;
+
+  /**
+   * @return true if the replica's metadata exists.
+   */
+  abstract public boolean metadataExists();
 
   /**
-   * This function "breaks hardlinks" to the current replica file.
+   * Used to deletes the replica's metadata.
    *
-   * When doing a DataNode upgrade, we create a bunch of hardlinks to each block
-   * file.  This cleverly ensures that both the old and the new storage
-   * directories can contain the same block file, without using additional space
-   * for the data.
+   * @return true if the replica's metadata is successfully deleted.
+   */
+  abstract public boolean deleteMetadata();
+
+  /**
+   * @return the length of the metadata on storage.
+   */
+  abstract public long getMetadataLength();
+
+  /**
+   * Rename the metadata {@link URI} to that referenced by {@code destURI}.
    *
-   * However, when we want to append to the replica file, we need to "break" the
-   * hardlink to ensure that the old snapshot continues to contain the old data
-   * length.  If we failed to do that, we could roll back to the previous/
-   * directory during a downgrade, and find that the block contents were longer
-   * than they were at the time of upgrade.
+   * @param destURI the target {@link URI}.
+   * @return true if the rename is successful.
+   * @throws IOException if an exception occurs in the rename.
+   */
+  abstract public boolean renameMeta(URI destURI) throws IOException;
+
+  /**
+   * Rename the data {@link URI} to that referenced by {@code destURI}.
    *
-   * @return true only if data was copied.
+   * @param destURI the target {@link URI}.
+   * @return true if the rename is successful.
+   * @throws IOException if an exception occurs in the rename.
+   */
+  abstract public boolean renameData(URI destURI) throws IOException;
+
+  /**
+   * Update this replica with the {@link StorageLocation} found.
+   * @param replicaLocation the {@link StorageLocation} found for this replica.
+   */
+  abstract public void updateWithReplica(StorageLocation replicaLocation);
+
+  /**
+   * Check whether the block was pinned.
+   * @param localFS the local filesystem to use.
+   * @return true if the block is pinned.
    * @throws IOException
    */
-  public boolean breakHardLinksIfNeeded() throws IOException {
-    File file = getBlockFile();
-    if (file == null || getVolume() == null) {
-      throw new IOException("detachBlock:Block not found. " + this);
-    }
-    File meta = getMetaFile();
-
-    int linkCount = HardLink.getLinkCount(file);
-    if (linkCount > 1) {
-      DataNode.LOG.info("Breaking hardlink for " + linkCount + "x-linked " +
-          "block " + this);
-      breakHardlinks(file, this);
-    }
-    if (HardLink.getLinkCount(meta) > 1) {
-      breakHardlinks(meta, this);
-    }
-    return true;
+  abstract public boolean getPinning(LocalFileSystem localFS)
+      throws IOException;
+
+  /**
+   * Set a block to be pinned on this datanode so that it cannot be moved
+   * by Balancer/Mover.
+   *
+   * @param localFS the local filesystem to use.
+   * @throws IOException if there is an exception in the pinning.
+   */
+  abstract public void setPinning(LocalFileSystem localFS) throws IOException;
+
+  /**
+   * Bump a replica's generation stamp to a new one.
+   * Its on-disk meta file name is renamed to be the new one too.
+   *
+   * @param newGS new generation stamp
+   * @throws IOException if the change fails
+   */
+  abstract public void bumpReplicaGS(long newGS) throws IOException;
+
+  abstract public ReplicaInfo getOriginalReplica();
+
+  /**
+   * Get the recovery id.
+   * @return the generation stamp that the replica will be bumped to
+   */
+  abstract public long getRecoveryID();
+
+  /**
+   * Set the recovery id.
+   * @param recoveryId the new recoveryId
+   */
+  abstract public void setRecoveryID(long recoveryId);
+
+  abstract public boolean breakHardLinksIfNeeded() throws IOException;
+
+  abstract public ReplicaRecoveryInfo createInfo();
+
+  abstract public int compareWith(ScanInfo info);
+
+  abstract public void truncateBlock(long newLength) throws IOException;
+
+  abstract public void copyMetadata(URI destination) throws IOException;
+
+  abstract public void copyBlockdata(URI destination) throws IOException;
+
+  /**
+   * Number of bytes originally reserved for this replica. The actual
+   * reservation is adjusted as data is written to disk.
+   *
+   * @return the number of bytes originally reserved for this replica.
+   */
+  public long getOriginalBytesReserved() {
+    return 0;
   }
 
   @Override  //Object
@@ -298,7 +266,7 @@ abstract public class ReplicaInfo extends Block
         + "\n  getBytesOnDisk()  = " + getBytesOnDisk()
         + "\n  getVisibleLength()= " + getVisibleLength()
         + "\n  getVolume()       = " + getVolume()
-        + "\n  getBlockFile()    = " + getBlockFile();
+        + "\n  getBlockURI()     = " + getBlockURI();
   }
 
   @Override

+ 12 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.File;
-
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -31,19 +29,19 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
  * A recovery with higher recovery id preempts recoveries with a lower id.
  *
  */
-public class ReplicaUnderRecovery extends ReplicaInfo {
-  private ReplicaInfo original; // the original replica that needs to be recovered
+public class ReplicaUnderRecovery extends LocalReplica {
+  private LocalReplica original; // original replica to be recovered
   private long recoveryId; // recovery id; it is also the generation stamp 
                            // that the replica will be bumped to after recovery
 
   public ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
-    super(replica, replica.getVolume(), replica.getDir());
+    super(replica, replica.getVolume(), ((LocalReplica)replica).getDir());
     if ( replica.getState() != ReplicaState.FINALIZED &&
          replica.getState() != ReplicaState.RBW &&
          replica.getState() != ReplicaState.RWR ) {
       throw new IllegalArgumentException("Cannot recover replica: " + replica);
     }
-    this.original = replica;
+    this.original = (LocalReplica) replica;
     this.recoveryId = recoveryId;
   }
 
@@ -53,22 +51,16 @@ public class ReplicaUnderRecovery extends ReplicaInfo {
    */
   public ReplicaUnderRecovery(ReplicaUnderRecovery from) {
     super(from);
-    this.original = from.getOriginalReplica();
+    this.original = (LocalReplica) from.getOriginalReplica();
     this.recoveryId = from.getRecoveryID();
   }
 
-  /** 
-   * Get the recovery id
-   * @return the generation stamp that the replica will be bumped to 
-   */
+  @Override
   public long getRecoveryID() {
     return recoveryId;
   }
 
-  /** 
-   * Set the recovery id
-   * @param recoveryId the new recoveryId
-   */
+  @Override
   public void setRecoveryID(long recoveryId) {
     if (recoveryId > this.recoveryId) {
       this.recoveryId = recoveryId;
@@ -82,6 +74,7 @@ public class ReplicaUnderRecovery extends ReplicaInfo {
    * Get the original replica that's under recovery
    * @return the original replica under recovery
    */
+  @Override
   public ReplicaInfo getOriginalReplica() {
     return original;
   }
@@ -120,9 +113,9 @@ public class ReplicaUnderRecovery extends ReplicaInfo {
   }
   
   @Override //ReplicaInfo
-  public void setDir(File dir) {
-    super.setDir(dir);
-    original.setDir(dir);
+  public void updateWithReplica(StorageLocation replicaLocation) {
+    super.updateWithReplica(replicaLocation);
+    original.updateWithReplica(replicaLocation);
   }
   
   @Override //ReplicaInfo
@@ -148,6 +141,7 @@ public class ReplicaUnderRecovery extends ReplicaInfo {
         + "\n  original=" + original;
   }
 
+  @Override
   public ReplicaRecoveryInfo createInfo() {
     return new ReplicaRecoveryInfo(original.getBlockId(), 
         original.getBytesOnDisk(), original.getGenerationStamp(),

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

@@ -22,6 +22,7 @@ import java.io.File;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 
 /**
  * This class represents a replica that is waiting to be recovered.
@@ -32,7 +33,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
  * client continues to write or be recovered as a result of
  * lease recovery.
  */
-public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
+public class ReplicaWaitingToBeRecovered extends LocalReplica {
 
   /**
    * Constructor
@@ -94,4 +95,28 @@ public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
   public String toString() {
     return super.toString();
   }
+
+  @Override
+  public ReplicaInfo getOriginalReplica() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getOriginalReplica");
+  }
+
+  @Override
+  public long getRecoveryID() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getRecoveryID");
+  }
+
+  @Override
+  public void setRecoveryID(long recoveryId) {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getRecoveryID");
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support createInfo");
+  }
 }

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

@@ -44,9 +44,8 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
@@ -230,10 +229,10 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   VolumeFailureSummary getVolumeFailureSummary();
 
   /** @return a list of finalized blocks for the given block pool. */
-  List<FinalizedReplica> getFinalizedBlocks(String bpid);
+  List<ReplicaInfo> getFinalizedBlocks(String bpid);
 
   /** @return a list of finalized blocks for the given block pool. */
-  List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
+  List<ReplicaInfo> getFinalizedBlocksOnPersistentStorage(String bpid);
 
   /**
    * Check whether the in-memory block record matches the block on the disk,
@@ -337,7 +336,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param temporary the temporary replica being converted
    * @return the result RBW
    */
-  ReplicaInPipelineInterface convertTemporaryToRbw(
+  ReplicaInPipeline convertTemporaryToRbw(
       ExtendedBlock temporary) throws IOException;
 
   /**

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

@@ -45,13 +45,13 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
-import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
@@ -309,14 +309,14 @@ class BlockPoolSlice {
     return rbwFile;
   }
 
-  File addFinalizedBlock(Block b, File f) throws IOException {
+  File addFinalizedBlock(Block b, ReplicaInfo replicaInfo) throws IOException {
     File blockDir = DatanodeUtil.idToBlockDir(finalizedDir, b.getBlockId());
     if (!blockDir.exists()) {
       if (!blockDir.mkdirs()) {
         throw new IOException("Failed to mkdirs " + blockDir);
       }
     }
-    File blockFile = FsDatasetImpl.moveBlockFiles(b, f, blockDir);
+    File blockFile = FsDatasetImpl.moveBlockFiles(b, replicaInfo, blockDir);
     File metaFile = FsDatasetUtil.getMetaFile(blockFile, b.getGenerationStamp());
     if (dfsUsage instanceof CachingGetSpaceUsed) {
       ((CachingGetSpaceUsed) dfsUsage).incDfsUsed(
@@ -329,16 +329,28 @@ class BlockPoolSlice {
    * Move a persisted replica from lazypersist directory to a subdirectory
    * under finalized.
    */
-  File activateSavedReplica(Block b, File metaFile, File blockFile)
-      throws IOException {
-    final File blockDir = DatanodeUtil.idToBlockDir(finalizedDir, b.getBlockId());
+  ReplicaInfo activateSavedReplica(ReplicaInfo replicaInfo,
+      RamDiskReplica replicaState) throws IOException {
+    File metaFile = replicaState.getSavedMetaFile();
+    File blockFile = replicaState.getSavedBlockFile();
+    final long blockId = replicaInfo.getBlockId();
+    final File blockDir = DatanodeUtil.idToBlockDir(finalizedDir, blockId);
     final File targetBlockFile = new File(blockDir, blockFile.getName());
     final File targetMetaFile = new File(blockDir, metaFile.getName());
     FileUtils.moveFile(blockFile, targetBlockFile);
     FsDatasetImpl.LOG.info("Moved " + blockFile + " to " + targetBlockFile);
     FileUtils.moveFile(metaFile, targetMetaFile);
     FsDatasetImpl.LOG.info("Moved " + metaFile + " to " + targetMetaFile);
-    return targetBlockFile;
+
+    ReplicaInfo newReplicaInfo =
+        new ReplicaBuilder(ReplicaState.FINALIZED)
+        .setBlockId(blockId)
+        .setLength(replicaInfo.getBytesOnDisk())
+        .setGenerationStamp(replicaInfo.getGenerationStamp())
+        .setFsVolume(replicaState.getLazyPersistVolume())
+        .setDirectoryToUse(targetBlockFile.getParentFile())
+        .build();
+    return newReplicaInfo;
   }
 
   void checkDirs() throws DiskErrorException {
@@ -461,9 +473,13 @@ class BlockPoolSlice {
     long blockId = block.getBlockId();
     long genStamp = block.getGenerationStamp();
     if (isFinalized) {
-      newReplica = new FinalizedReplica(blockId,
-          block.getNumBytes(), genStamp, volume, DatanodeUtil
-          .idToBlockDir(finalizedDir, blockId));
+      newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
+          .setBlockId(blockId)
+          .setLength(block.getNumBytes())
+          .setGenerationStamp(genStamp)
+          .setFsVolume(volume)
+          .setDirectoryToUse(DatanodeUtil.idToBlockDir(finalizedDir, blockId))
+          .build();
     } else {
       File file = new File(rbwDir, block.getBlockName());
       boolean loadRwr = true;
@@ -477,9 +493,15 @@ class BlockPoolSlice {
           // It didn't expire. Load the replica as a RBW.
           // We don't know the expected block length, so just use 0
           // and don't reserve any more space for writes.
-          newReplica = new ReplicaBeingWritten(blockId,
-              validateIntegrityAndSetLength(file, genStamp),
-              genStamp, volume, file.getParentFile(), null, 0);
+          newReplica = new ReplicaBuilder(ReplicaState.RBW)
+              .setBlockId(blockId)
+              .setLength(validateIntegrityAndSetLength(file, genStamp))
+              .setGenerationStamp(genStamp)
+              .setFsVolume(volume)
+              .setDirectoryToUse(file.getParentFile())
+              .setWriterThread(null)
+              .setBytesToReserve(0)
+              .build();
           loadRwr = false;
         }
         sc.close();
@@ -496,9 +518,13 @@ class BlockPoolSlice {
       }
       // Restart meta doesn't exist or expired.
       if (loadRwr) {
-        newReplica = new ReplicaWaitingToBeRecovered(blockId,
-            validateIntegrityAndSetLength(file, genStamp),
-            genStamp, volume, file.getParentFile());
+        ReplicaBuilder builder = new ReplicaBuilder(ReplicaState.RWR)
+            .setBlockId(blockId)
+            .setLength(validateIntegrityAndSetLength(file, genStamp))
+            .setGenerationStamp(genStamp)
+            .setFsVolume(volume)
+            .setDirectoryToUse(file.getParentFile());
+        newReplica = builder.build();
       }
     }
 
@@ -614,7 +640,7 @@ class BlockPoolSlice {
 
     // it's the same block so don't ever delete it, even if GS or size
     // differs.  caller should keep the one it just discovered on disk
-    if (replica1.getBlockFile().equals(replica2.getBlockFile())) {
+    if (replica1.getBlockURI().equals(replica2.getBlockURI())) {
       return null;
     }
     if (replica1.getGenerationStamp() != replica2.getGenerationStamp()) {
@@ -641,13 +667,11 @@ class BlockPoolSlice {
 
   private void deleteReplica(final ReplicaInfo replicaToDelete) {
     // Delete the files on disk. Failure here is okay.
-    final File blockFile = replicaToDelete.getBlockFile();
-    if (!blockFile.delete()) {
-      LOG.warn("Failed to delete block file " + blockFile);
+    if (!replicaToDelete.deleteBlockData()) {
+      LOG.warn("Failed to delete block file for replica " + replicaToDelete);
     }
-    final File metaFile = replicaToDelete.getMetaFile();
-    if (!metaFile.delete()) {
-      LOG.warn("Failed to delete meta file " + metaFile);
+    if (!replicaToDelete.deleteMetadata()) {
+      LOG.warn("Failed to delete meta file for replica " + replicaToDelete);
     }
   }
 

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
 import java.io.FileDescriptor;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -34,6 +35,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.io.IOUtils;
@@ -211,12 +214,12 @@ class FsDatasetAsyncDiskService {
    * Delete the block file and meta file from the disk asynchronously, adjust
    * dfsUsed statistics accordingly.
    */
-  void deleteAsync(FsVolumeReference volumeRef, File blockFile, File metaFile,
+  void deleteAsync(FsVolumeReference volumeRef, ReplicaInfo replicaToDelete,
       ExtendedBlock block, String trashDirectory) {
     LOG.info("Scheduling " + block.getLocalBlock()
-        + " file " + blockFile + " for deletion");
+        + " replica " + replicaToDelete + " for deletion");
     ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
-        volumeRef, blockFile, metaFile, block, trashDirectory);
+        volumeRef, replicaToDelete, block, trashDirectory);
     execute(((FsVolumeImpl) volumeRef.getVolume()).getCurrentDir(), deletionTask);
   }
   
@@ -227,19 +230,18 @@ class FsDatasetAsyncDiskService {
    *  files are deleted immediately.
    */
   class ReplicaFileDeleteTask implements Runnable {
-    final FsVolumeReference volumeRef;
-    final FsVolumeImpl volume;
-    final File blockFile;
-    final File metaFile;
-    final ExtendedBlock block;
-    final String trashDirectory;
-    
-    ReplicaFileDeleteTask(FsVolumeReference volumeRef, File blockFile,
-        File metaFile, ExtendedBlock block, String trashDirectory) {
+    private final FsVolumeReference volumeRef;
+    private final FsVolumeImpl volume;
+    private final ReplicaInfo replicaToDelete;
+    private final ExtendedBlock block;
+    private final String trashDirectory;
+
+    ReplicaFileDeleteTask(FsVolumeReference volumeRef,
+        ReplicaInfo replicaToDelete, ExtendedBlock block,
+        String trashDirectory) {
       this.volumeRef = volumeRef;
       this.volume = (FsVolumeImpl) volumeRef.getVolume();
-      this.blockFile = blockFile;
-      this.metaFile = metaFile;
+      this.replicaToDelete = replicaToDelete;
       this.block = block;
       this.trashDirectory = trashDirectory;
     }
@@ -248,15 +250,22 @@ class FsDatasetAsyncDiskService {
     public String toString() {
       // Called in AsyncDiskService.execute for displaying error messages.
       return "deletion of block " + block.getBlockPoolId() + " "
-          + block.getLocalBlock() + " with block file " + blockFile
-          + " and meta file " + metaFile + " from volume " + volume;
+          + block.getLocalBlock() + " with block file "
+          + replicaToDelete.getBlockURI() + " and meta file "
+          + replicaToDelete.getMetadataURI() + " from volume " + volume;
     }
 
     private boolean deleteFiles() {
-      return blockFile.delete() && (metaFile.delete() || !metaFile.exists());
+      return replicaToDelete.deleteBlockData() &&
+        (replicaToDelete.deleteMetadata() || !replicaToDelete.metadataExists());
     }
 
     private boolean moveFiles() {
+      if (trashDirectory == null) {
+        LOG.error("Trash dir for replica " + replicaToDelete + " is null");
+        return false;
+      }
+
       File trashDirFile = new File(trashDirectory);
       if (!trashDirFile.exists() && !trashDirFile.mkdirs()) {
         LOG.error("Failed to create trash directory " + trashDirectory);
@@ -264,20 +273,28 @@ class FsDatasetAsyncDiskService {
       }
 
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Moving files " + blockFile.getName() + " and " +
-            metaFile.getName() + " to trash.");
+        LOG.debug("Moving files " + replicaToDelete.getBlockURI() + " and " +
+            replicaToDelete.getMetadataURI() + " to trash.");
       }
 
-      File newBlockFile = new File(trashDirectory, blockFile.getName());
-      File newMetaFile = new File(trashDirectory, metaFile.getName());
-      return (blockFile.renameTo(newBlockFile) &&
-              metaFile.renameTo(newMetaFile));
+      final String blockName = replicaToDelete.getBlockName();
+      final long genstamp = replicaToDelete.getGenerationStamp();
+      File newBlockFile = new File(trashDirectory, blockName);
+      File newMetaFile = new File(trashDirectory,
+          DatanodeUtil.getMetaName(blockName, genstamp));
+      try {
+        return (replicaToDelete.renameData(newBlockFile.toURI()) &&
+                replicaToDelete.renameMeta(newMetaFile.toURI()));
+      } catch (IOException e) {
+        LOG.error("Error moving files to trash: " + replicaToDelete, e);
+      }
+      return false;
     }
 
     @Override
     public void run() {
-      final long blockLength = blockFile.length();
-      final long metaLength = metaFile.length();
+      final long blockLength = replicaToDelete.getBlockDataLength();
+      final long metaLength = replicaToDelete.getMetadataLength();
       boolean result;
 
       result = (trashDirectory == null) ? deleteFiles() : moveFiles();
@@ -286,7 +303,7 @@ class FsDatasetAsyncDiskService {
         LOG.warn("Unexpected error trying to "
             + (trashDirectory == null ? "delete" : "move")
             + " block " + block.getBlockPoolId() + " " + block.getLocalBlock()
-            + " at file " + blockFile + ". Ignored.");
+            + " at file " + replicaToDelete.getBlockURI() + ". Ignored.");
       } else {
         if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
           datanode.notifyNamenodeDeletedBlock(block, volume.getStorageID());
@@ -294,7 +311,7 @@ class FsDatasetAsyncDiskService {
         volume.onBlockFileDeletion(block.getBlockPoolId(), blockLength);
         volume.onMetaFileDeletion(block.getBlockPoolId(), metaLength);
         LOG.info("Deleted " + block.getBlockPoolId() + " "
-            + block.getLocalBlock() + " file " + blockFile);
+            + block.getLocalBlock() + " URI " + replicaToDelete.getBlockURI());
       }
       updateDeletedBlockId(block);
       IOUtils.cleanup(null, volumeRef);

File diff suppressed because it is too large
+ 199 - 369
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java


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

@@ -18,14 +18,17 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.util.Arrays;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.io.IOUtils;
 
 /** Utility methods. */
 @InterfaceAudience.Private
@@ -71,6 +74,21 @@ public class FsDatasetUtil {
     return matches[0];
   }
 
+  public static FileInputStream openAndSeek(File file, long offset)
+      throws IOException {
+    RandomAccessFile raf = null;
+    try {
+      raf = new RandomAccessFile(file, "r");
+      if (offset > 0) {
+        raf.seek(offset);
+      }
+      return new FileInputStream(raf.getFD());
+    } catch(IOException ioe) {
+      IOUtils.cleanup(null, raf);
+      throw ioe;
+    }
+  }
+
   /**
    * Find the meta-file for the specified block file
    * and then return the generation stamp from the name of the meta-file.

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

@@ -47,11 +47,19 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.LocalReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
+import org.apache.hadoop.hdfs.server.datanode.LocalReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -102,7 +110,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   // Disk space reserved for blocks (RBW or Re-replicating) open for write.
   private AtomicLong reservedForReplicas;
   private long recentReserved = 0;
-
+  private final Configuration conf;
   // Capacity configured. This is useful when we want to
   // limit the visible capacity for tests. If negative, then we just
   // query from the filesystem.
@@ -130,6 +138,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
     this.usage = new DF(parent, conf);
     this.storageType = storageType;
     this.configuredCapacity = -1;
+    this.conf = conf;
     cacheExecutor = initializeCacheExecutor(parent);
   }
 
@@ -896,10 +905,15 @@ public class FsVolumeImpl implements FsVolumeSpi {
    * @return
    * @throws IOException
    */
-  File addFinalizedBlock(String bpid, Block b, File f, long bytesReserved)
-      throws IOException {
+  ReplicaInfo addFinalizedBlock(String bpid, Block b, ReplicaInfo replicaInfo,
+      long bytesReserved) throws IOException {
     releaseReservedSpace(bytesReserved);
-    return getBlockPoolSlice(bpid).addFinalizedBlock(b, f);
+    File dest = getBlockPoolSlice(bpid).addFinalizedBlock(b, replicaInfo);
+    return new ReplicaBuilder(ReplicaState.FINALIZED)
+        .setBlock(replicaInfo)
+        .setFsVolume(this)
+        .setDirectoryToUse(dest.getParentFile())
+        .build();
   }
 
   Executor getCacheExecutor() {
@@ -950,18 +964,18 @@ public class FsVolumeImpl implements FsVolumeSpi {
     }
   }
 
-  void addBlockPool(String bpid, Configuration conf) throws IOException {
-    addBlockPool(bpid, conf, null);
+  void addBlockPool(String bpid, Configuration c) throws IOException {
+    addBlockPool(bpid, c, null);
   }
 
-  void addBlockPool(String bpid, Configuration conf, Timer timer)
+  void addBlockPool(String bpid, Configuration c, Timer timer)
       throws IOException {
     File bpdir = new File(currentDir, bpid);
     BlockPoolSlice bp;
     if (timer == null) {
-      bp = new BlockPoolSlice(bpid, this, bpdir, conf, new Timer());
+      bp = new BlockPoolSlice(bpid, this, bpdir, c, new Timer());
     } else {
-      bp = new BlockPoolSlice(bpid, this, bpdir, conf, timer);
+      bp = new BlockPoolSlice(bpid, this, bpdir, c, timer);
     }
     bpSlices.put(bpid, bp);
   }
@@ -1053,5 +1067,127 @@ public class FsVolumeImpl implements FsVolumeSpi {
   DatanodeStorage toDatanodeStorage() {
     return new DatanodeStorage(storageID, DatanodeStorage.State.NORMAL, storageType);
   }
+
+
+  public ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo,
+      long newGS, long estimateBlockLen) throws IOException {
+
+    long bytesReserved = estimateBlockLen - replicaInfo.getNumBytes();
+    if (getAvailable() < bytesReserved) {
+      throw new DiskOutOfSpaceException("Insufficient space for appending to "
+          + replicaInfo);
+    }
+
+    assert replicaInfo.getVolume() == this:
+      "The volume of the replica should be the same as this volume";
+
+    // construct a RBW replica with the new GS
+    File newBlkFile = new File(getRbwDir(bpid), replicaInfo.getBlockName());
+    LocalReplicaInPipeline newReplicaInfo = new ReplicaBuilder(ReplicaState.RBW)
+        .setBlockId(replicaInfo.getBlockId())
+        .setLength(replicaInfo.getNumBytes())
+        .setGenerationStamp(newGS)
+        .setFsVolume(this)
+        .setDirectoryToUse(newBlkFile.getParentFile())
+        .setWriterThread(Thread.currentThread())
+        .setBytesToReserve(bytesReserved)
+        .buildLocalReplicaInPipeline();
+
+    // rename meta file to rbw directory
+    // rename block file to rbw directory
+    newReplicaInfo.moveReplicaFrom(replicaInfo, newBlkFile);
+
+    reserveSpaceForReplica(bytesReserved);
+    return newReplicaInfo;
+  }
+
+  public ReplicaInPipeline createRbw(ExtendedBlock b) throws IOException {
+
+    File f = createRbwFile(b.getBlockPoolId(), b.getLocalBlock());
+    LocalReplicaInPipeline newReplicaInfo = new ReplicaBuilder(ReplicaState.RBW)
+        .setBlockId(b.getBlockId())
+        .setGenerationStamp(b.getGenerationStamp())
+        .setFsVolume(this)
+        .setDirectoryToUse(f.getParentFile())
+        .setBytesToReserve(b.getNumBytes())
+        .buildLocalReplicaInPipeline();
+    return newReplicaInfo;
+  }
+
+  public ReplicaInPipeline convertTemporaryToRbw(ExtendedBlock b,
+      ReplicaInfo temp) throws IOException {
+
+    final long blockId = b.getBlockId();
+    final long expectedGs = b.getGenerationStamp();
+    final long visible = b.getNumBytes();
+    final long numBytes = temp.getNumBytes();
+
+    // move block files to the rbw directory
+    BlockPoolSlice bpslice = getBlockPoolSlice(b.getBlockPoolId());
+    final File dest = FsDatasetImpl.moveBlockFiles(b.getLocalBlock(), temp,
+        bpslice.getRbwDir());
+    // create RBW
+    final LocalReplicaInPipeline rbw = new ReplicaBuilder(ReplicaState.RBW)
+        .setBlockId(blockId)
+        .setLength(numBytes)
+        .setGenerationStamp(expectedGs)
+        .setFsVolume(this)
+        .setDirectoryToUse(dest.getParentFile())
+        .setWriterThread(Thread.currentThread())
+        .setBytesToReserve(0)
+        .buildLocalReplicaInPipeline();
+    rbw.setBytesAcked(visible);
+    return rbw;
+  }
+
+  public ReplicaInPipeline createTemporary(ExtendedBlock b) throws IOException {
+    // create a temporary file to hold block in the designated volume
+    File f = createTmpFile(b.getBlockPoolId(), b.getLocalBlock());
+    LocalReplicaInPipeline newReplicaInfo =
+        new ReplicaBuilder(ReplicaState.TEMPORARY)
+          .setBlockId(b.getBlockId())
+          .setGenerationStamp(b.getGenerationStamp())
+          .setDirectoryToUse(f.getParentFile())
+          .setBytesToReserve(b.getLocalBlock().getNumBytes())
+          .setFsVolume(this)
+          .buildLocalReplicaInPipeline();
+    return newReplicaInfo;
+  }
+
+  public ReplicaInPipeline updateRURCopyOnTruncate(ReplicaInfo rur,
+      String bpid, long newBlockId, long recoveryId, long newlength)
+      throws IOException {
+
+    rur.breakHardLinksIfNeeded();
+    File[] copiedReplicaFiles =
+        copyReplicaWithNewBlockIdAndGS(rur, bpid, newBlockId, recoveryId);
+    File blockFile = copiedReplicaFiles[1];
+    File metaFile = copiedReplicaFiles[0];
+    LocalReplica.truncateBlock(blockFile, metaFile,
+        rur.getNumBytes(), newlength);
+
+    LocalReplicaInPipeline newReplicaInfo = new ReplicaBuilder(ReplicaState.RBW)
+        .setBlockId(newBlockId)
+        .setGenerationStamp(recoveryId)
+        .setFsVolume(this)
+        .setDirectoryToUse(blockFile.getParentFile())
+        .setBytesToReserve(newlength)
+        .buildLocalReplicaInPipeline();
+    return newReplicaInfo;
+  }
+
+  private File[] copyReplicaWithNewBlockIdAndGS(
+      ReplicaInfo replicaInfo, String bpid, long newBlkId, long newGS)
+      throws IOException {
+    String blockFileName = Block.BLOCK_FILE_PREFIX + newBlkId;
+    FsVolumeImpl v = (FsVolumeImpl) replicaInfo.getVolume();
+    final File tmpDir = v.getBlockPoolSlice(bpid).getTmpDir();
+    final File destDir = DatanodeUtil.idToBlockDir(tmpDir, newBlkId);
+    final File dstBlockFile = new File(destDir, blockFileName);
+    final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
+    return FsDatasetImpl.copyBlockFiles(replicaInfo, dstMetaFile,
+        dstBlockFile, true, DFSUtilClient.getSmallBufferSize(conf), conf);
+  }
+
 }
 

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

@@ -201,7 +201,7 @@ class FsVolumeList {
                 + " " + bpid + " on volume " + v + ": " + timeTaken + "ms");
           } catch (ClosedChannelException e) {
             FsDatasetImpl.LOG.info("The volume " + v + " is closed while " +
-                "addng replicas, ignored.");
+                "adding replicas, ignored.");
           } catch (IOException ioe) {
             FsDatasetImpl.LOG.info("Caught exception while adding replicas " +
                 "from " + v + ". Will throw later.", ioe);
@@ -311,7 +311,7 @@ class FsVolumeList {
     } else {
       // If the volume is not put into a volume scanner, it does not need to
       // hold the reference.
-      IOUtils.cleanup(FsDatasetImpl.LOG, ref);
+      IOUtils.cleanup(null, ref);
     }
     // If the volume is used to replace a failed volume, it needs to reset the
     // volume failure info for this volume.

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

@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 
 import java.io.File;
@@ -182,8 +183,7 @@ class RamDiskAsyncLazyPersistService {
    */
   void submitLazyPersistTask(String bpId, long blockId,
       long genStamp, long creationTime,
-      File metaFile, File blockFile,
-      FsVolumeReference target) throws IOException {
+      ReplicaInfo replica, FsVolumeReference target) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("LazyWriter schedule async task to persist RamDisk block pool id: "
           + bpId + " block id: " + blockId);
@@ -198,31 +198,29 @@ class RamDiskAsyncLazyPersistService {
     }
 
     ReplicaLazyPersistTask lazyPersistTask = new ReplicaLazyPersistTask(
-        bpId, blockId, genStamp, creationTime, blockFile, metaFile,
+        bpId, blockId, genStamp, creationTime, replica,
         target, lazyPersistDir);
     execute(volume.getCurrentDir(), lazyPersistTask);
   }
 
   class ReplicaLazyPersistTask implements Runnable {
-    final String bpId;
-    final long blockId;
-    final long genStamp;
-    final long creationTime;
-    final File blockFile;
-    final File metaFile;
-    final FsVolumeReference targetVolume;
-    final File lazyPersistDir;
+    private final String bpId;
+    private final long blockId;
+    private final long genStamp;
+    private final long creationTime;
+    private final ReplicaInfo replicaInfo;
+    private final FsVolumeReference targetVolume;
+    private final File lazyPersistDir;
 
     ReplicaLazyPersistTask(String bpId, long blockId,
         long genStamp, long creationTime,
-        File blockFile, File metaFile,
+        ReplicaInfo replicaInfo,
         FsVolumeReference targetVolume, File lazyPersistDir) {
       this.bpId = bpId;
       this.blockId = blockId;
       this.genStamp = genStamp;
       this.creationTime = creationTime;
-      this.blockFile = blockFile;
-      this.metaFile = metaFile;
+      this.replicaInfo = replicaInfo;
       this.targetVolume = targetVolume;
       this.lazyPersistDir = lazyPersistDir;
     }
@@ -232,8 +230,10 @@ class RamDiskAsyncLazyPersistService {
       // Called in AsyncLazyPersistService.execute for displaying error messages.
       return "LazyWriter async task of persist RamDisk block pool id:"
           + bpId + " block pool id: "
-          + blockId + " with block file " + blockFile
-          + " and meta file " + metaFile + " to target volume " + targetVolume;}
+          + blockId + " with block file " + replicaInfo.getBlockURI()
+          + " and meta file " + replicaInfo.getMetadataURI()
+          + " to target volume " + targetVolume;
+    }
 
     @Override
     public void run() {
@@ -243,7 +243,7 @@ class RamDiskAsyncLazyPersistService {
         int smallBufferSize = DFSUtilClient.getSmallBufferSize(EMPTY_HDFS_CONF);
         // No FsDatasetImpl lock for the file copy
         File targetFiles[] = FsDatasetImpl.copyBlockFiles(
-            blockId, genStamp, metaFile, blockFile, lazyPersistDir, true,
+            blockId, genStamp, replicaInfo, lazyPersistDir, true,
             smallBufferSize, conf);
 
         // Lock FsDataSetImpl during onCompleteLazyPersist callback

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

@@ -326,7 +326,7 @@ public class FSDirectory implements Closeable {
     int threshold = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY,
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT);
-    NameNode.LOG.info("Caching file names occuring more than " + threshold
+    NameNode.LOG.info("Caching file names occurring more than " + threshold
         + " times");
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;

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

@@ -245,7 +245,7 @@ public class NNStorage extends Storage implements Closeable,
     /* We don't want more than one thread trying to restore at a time */
     synchronized (this.restorationLock) {
       LOG.info("NNStorage.attemptRestoreRemovedStorage: check removed(failed) "+
-               "storarge. removedStorages size = " + removedStorageDirs.size());
+               "storage. removedStorages size = " + removedStorageDirs.size());
       for (StorageDirectory sd : this.removedStorageDirs) {
         File root = sd.getRoot();
         LOG.info("currently disabled dir " + root.getAbsolutePath() +

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -168,7 +168,6 @@ import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
-import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.ipc.RefreshRegistry;
 import org.apache.hadoop.ipc.RefreshResponse;
 import org.apache.hadoop.net.Node;
@@ -317,8 +316,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         new TraceAdminProtocolServerSideTranslatorPB(this);
     BlockingService traceAdminService = TraceAdminService
         .newReflectiveBlockingService(traceAdminXlator);
-    
-    WritableRpcEngine.ensureInitialized();
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     if (serviceRpcAddr != null) {

+ 10 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java

@@ -135,13 +135,20 @@ public class DiskBalancerCLI extends Configured implements Tool {
   private static final Options CANCEL_OPTIONS = new Options();
   private static final Options REPORT_OPTIONS = new Options();
 
+  private final PrintStream printStream;
+
   /**
    * Construct a DiskBalancer.
    *
    * @param conf
    */
   public DiskBalancerCLI(Configuration conf) {
+    this(conf, System.out);
+  }
+
+  public DiskBalancerCLI(Configuration conf, final PrintStream printStream) {
     super(conf);
+    this.printStream = printStream;
   }
 
   /**
@@ -171,21 +178,9 @@ public class DiskBalancerCLI extends Configured implements Tool {
    */
   @Override
   public int run(String[] args) throws Exception {
-    return run(args, System.out);
-  }
-
-  /**
-   * Execute the command with the given arguments.
-   *
-   * @param args command specific arguments.
-   * @param out  the output stream used for printing
-   * @return exit code.
-   * @throws Exception
-   */
-  public int run(String[] args, final PrintStream out) throws Exception {
     Options opts = getOpts();
     CommandLine cmd = parseArgs(args, opts);
-    return dispatch(cmd, opts, out);
+    return dispatch(cmd, opts);
   }
 
   /**
@@ -443,7 +438,7 @@ public class DiskBalancerCLI extends Configured implements Tool {
    * @param opts options of command line
    * @param out  the output stream used for printing
    */
-  private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
+  private int dispatch(CommandLine cmd, Options opts)
       throws Exception {
     Command currentCommand = null;
     if (cmd.hasOption(DiskBalancerCLI.PLAN)) {
@@ -463,7 +458,7 @@ public class DiskBalancerCLI extends Configured implements Tool {
     }
 
     if (cmd.hasOption(DiskBalancerCLI.REPORT)) {
-      currentCommand = new ReportCommand(getConf(), out);
+      currentCommand = new ReportCommand(getConf(), this.printStream);
     }
 
     if (cmd.hasOption(DiskBalancerCLI.HELP)) {

+ 22 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -232,32 +232,42 @@ public class JsonUtil {
     return m;
   }
 
+  private static Map<String, Object> toJson(final DirectoryListing listing)
+      throws IOException {
+    final Map<String, Object> m = new TreeMap<>();
+    // Serialize FileStatus[] to a FileStatuses map
+    m.put("partialListing", toJsonMap(listing.getPartialListing()));
+    // Simple int
+    m.put("remainingEntries", listing.getRemainingEntries());
+
+    return m;
+  }
+
   public static String toJsonString(final DirectoryListing listing) throws
       IOException {
 
     if (listing == null) {
       return null;
     }
-
-    final Map<String, Object> m = new TreeMap<>();
-    m.put("partialListing", toJsonArray(listing.getPartialListing()));
-    m.put("remainingEntries", listing.getRemainingEntries());
-    return MAPPER.writeValueAsString(m);
+    return toJsonString(DirectoryListing.class, toJson(listing));
   }
 
-  private static Object[] toJsonArray(HdfsFileStatus[] statuses) throws
+  private static Map<String, Object> toJsonMap(HdfsFileStatus[] statuses) throws
       IOException {
     if (statuses == null) {
       return null;
     }
-    if (statuses.length == 0) {
-      return EMPTY_OBJECT_ARRAY;
-    }
-    final Object[] a = new Object[statuses.length];
+
+    final Map<String, Object> fileStatuses = new TreeMap<>();
+    final Map<String, Object> fileStatus = new TreeMap<>();
+    fileStatuses.put("FileStatuses", fileStatus);
+    final Object[] array = new Object[statuses.length];
+    fileStatus.put("FileStatus", array);
     for (int i = 0; i < statuses.length; i++) {
-      a[i] = toJsonMap(statuses[i]);
+      array[i] = toJsonMap(statuses[i]);
     }
-    return a;
+
+    return fileStatuses;
   }
 
   /** Convert a LocatedBlock[] to a Json array. */

+ 108 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md

@@ -19,6 +19,7 @@ HDFS Disk Balancer
 * [Architecture](#Architecture)
 * [Commands](#Commands)
 * [Settings](#Settings)
+* [Debugging](#Debugging)
 
 
 Overview
@@ -67,9 +68,12 @@ The following sections discusses what commands are supported by disk balancer
 | `-thresholdPercentage`| Since we operate against a snap-shot of datanode, the move operations have a tolerance percentage to declare success. If user specifies 10% and move operation is say 20GB in size, if we can move 18GB that operation is considered successful. This is to accommodate the changes in datanode in real time. This parameter is not needed and a default is used if not specified.|
 | `-maxerror` | Max error allows users to specify how many block copy operations must fail before we abort a move step. Once again, this is not a needed parameter and a system-default is used if not specified.|
 | `-v`| Verbose mode, specifying this parameter forces the plan command to print out a summary of the plan on stdout.|
+|`-fs`| - Specifies the namenode to use. if not specified default from config  is used. |
+
 
 The plan command writes two output files. They are `<nodename>.before.json` which
-captures the state of the datanode before the diskbalancer is run, and `<nodename>.plan.json`.
+captures the state of the cluster before the diskbalancer is run, and
+`<nodename>.plan.json`.
 
 ### Execute
 
@@ -118,3 +122,106 @@ There is a set of diskbalancer settings that can be controlled via hdfs-site.xml
 |`dfs.disk.balancer.max.disk.throughputInMBperSec` | This controls the maximum disk bandwidth consumed by diskbalancer while copying data. If a value like 10MB is specified then diskbalancer on the average will only copy 10MB/S. The default value is 10MB/S.|
 |`dfs.disk.balancer.max.disk.errors`| sets the value of maximum number of errors we can ignore for a specific move between two disks before it is abandoned. For example, if a plan has 3 pair of disks to copy between , and the first disk set encounters more than 5 errors, then we abandon the first copy and start the second copy in the plan. The default value of max errors is set to 5.|
 |`dfs.disk.balancer.block.tolerance.percent`| The tolerance percent specifies when we have reached a good enough value for any copy step. For example, if you specify 10% then getting close to 10% of the target value is good enough.|
+
+ Debugging
+---------
+
+Disk balancer generates two output files. The nodename.before.json contains
+the state of cluster that we read from the  namenode. This file
+contains detailed information about  datanodes and volumes.
+
+if you plan to post this file to an apache JIRA, you might want to
+replace your hostnames and volume paths since it may leak your personal
+information.
+
+You can also trim this file down to focus only on the nodes that you want to
+report in the JIRA.
+
+The nodename.plan.json contains the plan for the specific node. This plan
+file contains as a series of steps. A step is executed as a series of move
+operations inside the datanode.
+
+To diff the state of a node before and after, you can either re-run a plan
+command and diff the new nodename.before.json with older before.json or run
+report command against the node.
+
+To see the progress of a running plan, please run query command with option -v.
+This will print out a set of steps -- Each step represents a move operation
+from one disk to another.
+
+The speed of move is limited by the bandwidth that is specified. The default
+value of bandwidth is set to 10MB/sec. if you do a query with -v option you
+will see the following values.
+
+
+      "sourcePath" : "/data/disk2/hdfs/dn",
+
+      "destPath" : "/data/disk3/hdfs/dn",
+
+      "workItem" :
+
+        "startTime" : 1466575335493,
+
+        "secondsElapsed" : 16486,
+
+        "bytesToCopy" : 181242049353,
+
+        "bytesCopied" : 172655116288,
+
+        "errorCount" : 0,
+
+        "errMsg" : null,
+
+        "blocksCopied" : 1287,
+
+        "maxDiskErrors" : 5,
+
+        "tolerancePercent" : 10,
+
+        "bandwidth" : 10
+
+
+ *source path* - is the volume we are copying from.
+
+ *dest path* - is the volume to where we are copying to.
+
+ *start time* - is current time in milliseconds.
+
+ *seconds elapsed* - is updated whenever we update the stats. This might be
+ slower than the wall clock time.
+
+ *bytes to copy* - is number of bytes we are supposed to copy. We copy plus or
+ minus a certain percentage. So often you will see bytesCopied -- as a value
+ lesser than bytes to copy. In the default case, getting within 10% of bytes
+ to move is considered good enough.
+
+ *bytes copied* - is the actual number of bytes that we moved from source disk to
+ destination disk.
+
+ *error count* - Each time we encounter an error we will increment the error
+ count. As long as error count remains less than max error count (default
+ value is 5), we will try to complete this move. if we hit the max error count
+ we will abandon this current step and execute the next step in the plan.
+
+ *error message* - Currently a single string that reports the last error message.
+ Older messages should be in the datanode log.
+
+
+ *blocks copied* - Number of blocks copied.
+
+ *max disk errors* - The configuration used for this move step. currently it will
+ report the default config value, since the user interface to control these
+ values per step is not in place. It is a future work item. The default or
+ the command line value specified in plan command is used for this value.
+
+*tolerance percent* - This represents how much off we can be while moving data.
+In a busy cluster this allows admin to say, compute a plan, but I know this
+node is being used so it is okay if disk balancer can reach +/- 10% of the
+bytes to be copied.
+
+*bandwidth* - This is the maximum aggregate source disk bandwidth used by the
+disk balancer. After moving a block disk balancer computes how many seconds it
+should have taken to move that block with the specified bandwidth. If the
+actual move took less time than expected, then disk balancer will sleep for
+that duration. Please note that currently all moves are executed
+sequentially by a single thread.

+ 122 - 61
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -597,15 +597,15 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatu
 
         curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS_BATCH&startAfter=<CHILD>"
 
-    The client receives a response with a batch of [`FileStatuses` JSON object](#FileStatuses_JSON_Schema), as well as iteration information:
+    The client receives a response with a [`DirectoryListing` JSON object](#DirectoryListing_JSON_Schema), which contains a [`FileStatuses` JSON object](#FileStatuses_JSON_Schema), as well as iteration information:
 
         HTTP/1.1 200 OK
         Cache-Control: no-cache
-        Expires: Tue, 30 Aug 2016 16:42:16 GMT
-        Date: Tue, 30 Aug 2016 16:42:16 GMT
+        Expires: Thu, 08 Sep 2016 03:40:38 GMT
+        Date: Thu, 08 Sep 2016 03:40:38 GMT
         Pragma: no-cache
-        Expires: Tue, 30 Aug 2016 16:42:16 GMT
-        Date: Tue, 30 Aug 2016 16:42:16 GMT
+        Expires: Thu, 08 Sep 2016 03:40:38 GMT
+        Date: Thu, 08 Sep 2016 03:40:38 GMT
         Pragma: no-cache
         Content-Type: application/json
         X-FRAME-OPTIONS: SAMEORIGIN
@@ -613,56 +613,61 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatu
         Server: Jetty(6.1.26)
 
         {
-            "partialListing": [
-                {
-                    "accessTime": 0,
-                    "blockSize": 0,
-                    "childrenNum": 0,
-                    "fileId": 16389,
-                    "group": "supergroup",
-                    "length": 0,
-                    "modificationTime": 1472575493064,
-                    "owner": "andrew",
-                    "pathSuffix": "anotherdir",
-                    "permission": "755",
-                    "replication": 0,
-                    "storagePolicy": 0,
-                    "type": "DIRECTORY"
+            "DirectoryListing": {
+                "partialListing": {
+                    "FileStatuses": {
+                        "FileStatus": [
+                            {
+                                "accessTime": 0,
+                                "blockSize": 0,
+                                "childrenNum": 0,
+                                "fileId": 16387,
+                                "group": "supergroup",
+                                "length": 0,
+                                "modificationTime": 1473305882563,
+                                "owner": "andrew",
+                                "pathSuffix": "bardir",
+                                "permission": "755",
+                                "replication": 0,
+                                "storagePolicy": 0,
+                                "type": "DIRECTORY"
+                            },
+                            {
+                                "accessTime": 1473305896945,
+                                "blockSize": 1024,
+                                "childrenNum": 0,
+                                "fileId": 16388,
+                                "group": "supergroup",
+                                "length": 0,
+                                "modificationTime": 1473305896965,
+                                "owner": "andrew",
+                                "pathSuffix": "bazfile",
+                                "permission": "644",
+                                "replication": 3,
+                                "storagePolicy": 0,
+                                "type": "FILE"
+                            }
+                        ]
+                    }
                 },
-                {
-                    "accessTime": 0,
-                    "blockSize": 0,
-                    "childrenNum": 0,
-                    "fileId": 16386,
-                    "group": "supergroup",
-                    "length": 0,
-                    "modificationTime": 1472575274776,
-                    "owner": "andrew",
-                    "pathSuffix": "somedir",
-                    "permission": "755",
-                    "replication": 0,
-                    "storagePolicy": 0,
-                    "type": "DIRECTORY"
-                }
-            ],
-            "remainingEntries": 1
+                "remainingEntries": 2
+            }
         }
 
-
 If `remainingEntries` is non-zero, there are additional entries in the directory.
 To query the next batch, set the `startAfter` parameter to the `pathSuffix` of the last item returned in the current batch. For example:
 
-        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS_BATCH&startAfter=somedir"
+        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS_BATCH&startAfter=bazfile"
 
 Which will return the next batch of directory entries:
 
         HTTP/1.1 200 OK
         Cache-Control: no-cache
-        Expires: Tue, 30 Aug 2016 16:46:23 GMT
-        Date: Tue, 30 Aug 2016 16:46:23 GMT
+        Expires: Thu, 08 Sep 2016 03:43:20 GMT
+        Date: Thu, 08 Sep 2016 03:43:20 GMT
         Pragma: no-cache
-        Expires: Tue, 30 Aug 2016 16:46:23 GMT
-        Date: Tue, 30 Aug 2016 16:46:23 GMT
+        Expires: Thu, 08 Sep 2016 03:43:20 GMT
+        Date: Thu, 08 Sep 2016 03:43:20 GMT
         Pragma: no-cache
         Content-Type: application/json
         X-FRAME-OPTIONS: SAMEORIGIN
@@ -670,24 +675,45 @@ Which will return the next batch of directory entries:
         Server: Jetty(6.1.26)
 
         {
-            "partialListing": [
-                {
-                    "accessTime": 1472575333568,
-                    "blockSize": 1024,
-                    "childrenNum": 0,
-                    "fileId": 16388,
-                    "group": "supergroup",
-                    "length": 224,
-                    "modificationTime": 1472575334222,
-                    "owner": "andrew",
-                    "pathSuffix": "somefile",
-                    "permission": "644",
-                    "replication": 3,
-                    "storagePolicy": 0,
-                    "type": "FILE"
-                }
-            ],
-            "remainingEntries": 0
+            "DirectoryListing": {
+                "partialListing": {
+                    "FileStatuses": {
+                        "FileStatus": [
+                            {
+                                "accessTime": 0,
+                                "blockSize": 0,
+                                "childrenNum": 0,
+                                "fileId": 16386,
+                                "group": "supergroup",
+                                "length": 0,
+                                "modificationTime": 1473305878951,
+                                "owner": "andrew",
+                                "pathSuffix": "foodir",
+                                "permission": "755",
+                                "replication": 0,
+                                "storagePolicy": 0,
+                                "type": "DIRECTORY"
+                            },
+                            {
+                                "accessTime": 1473305902864,
+                                "blockSize": 1024,
+                                "childrenNum": 0,
+                                "fileId": 16389,
+                                "group": "supergroup",
+                                "length": 0,
+                                "modificationTime": 1473305902878,
+                                "owner": "andrew",
+                                "pathSuffix": "quxfile",
+                                "permission": "644",
+                                "replication": 3,
+                                "storagePolicy": 0,
+                                "type": "FILE"
+                            }
+                        ]
+                    }
+                },
+                "remainingEntries": 0
+            }
         }
 
 Batch size is controlled by the `dfs.ls.limit` option on the NameNode.
@@ -1672,6 +1698,41 @@ A `FileStatuses` JSON object represents an array of `FileStatus` JSON objects.
 
 See also: [`FileStatus` Properties](#FileStatus_Properties), [`LISTSTATUS`](#List_a_Directory), [FileStatus](../../api/org/apache/hadoop/fs/FileStatus.html)
 
+### DirectoryListing JSON Schema
+
+A `DirectoryListing` JSON object represents a batch of directory entries while iteratively listing a directory. It contains a `FileStatuses` JSON object as well as iteration information.
+
+```json
+{
+  "name"      : "DirectoryListing",
+  "properties":
+  {
+    "DirectoryListing":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "partialListing":
+        {
+          "description": "A partial directory listing",
+          "type"       : "object", // A FileStatuses object
+          "required"   : true
+        },
+        "remainingEntries":
+        {
+          "description": "Number of remaining entries",
+          "type"       : "integer",
+          "required"   : true
+        }
+      }
+    }
+  }
+
+}
+```
+
+See also: [`FileStatuses` JSON Schema](#FileStatuses_JSON_Schema), [`LISTSTATUS_BATCH`](#Iteratively_List_a_Directory), [FileStatus](../../api/org/apache/hadoop/fs/FileStatus.html)
+
 ### Long JSON Schema
 
 ```json

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -238,7 +239,8 @@ public class AppendTestUtil {
   }
 
   public static void testAppend(FileSystem fs, Path p) throws IOException {
-    final byte[] bytes = new byte[1000];
+    final int size = 1000;
+    final byte[] bytes = randomBytes(seed, size);
 
     { //create file
       final FSDataOutputStream out = fs.create(p, (short)1);
@@ -247,12 +249,22 @@ public class AppendTestUtil {
       assertEquals(bytes.length, fs.getFileStatus(p).getLen());
     }
 
-    for(int i = 2; i < 500; i++) {
+    final int appends = 50;
+    for (int i = 2; i < appends; i++) {
       //append
       final FSDataOutputStream out = fs.append(p);
       out.write(bytes);
       out.close();
-      assertEquals(i*bytes.length, fs.getFileStatus(p).getLen());
+      assertEquals(i * bytes.length, fs.getFileStatus(p).getLen());
     }
+
+    // Check the appended content
+    final FSDataInputStream in = fs.open(p);
+    for (int i = 0; i < appends - 1; i++) {
+      byte[] read = new byte[size];
+      in.read(i * bytes.length, read, 0, size);
+      assertArrayEquals(bytes, read);
+    }
+    in.close();
   }
 }

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

@@ -39,7 +39,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -530,7 +530,7 @@ public class TestClientProtocolForPipelineRecovery {
 
       DataNodeFaultInjector.set(new DataNodeFaultInjector() {
         @Override
-        public void failPipeline(ReplicaInPipelineInterface replicaInfo,
+        public void failPipeline(ReplicaInPipeline replicaInfo,
             String mirror) throws IOException {
           if (!lastDn.equals(mirror)) {
             // Only fail for second DN

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Before;
 import org.junit.Test;
@@ -172,12 +172,12 @@ public class TestCrcCorruption {
       final int dnIdx = 0;
       final DataNode dn = cluster.getDataNodes().get(dnIdx);
       final String bpid = cluster.getNamesystem().getBlockPoolId();
-      List<FinalizedReplica> replicas =
+      List<ReplicaInfo> replicas =
           dn.getFSDataset().getFinalizedBlocks(bpid);
       assertTrue("Replicas do not exist", !replicas.isEmpty());
 
       for (int idx = 0; idx < replicas.size(); idx++) {
-        FinalizedReplica replica = replicas.get(idx);
+        ReplicaInfo replica = replicas.get(idx);
         ExtendedBlock eb = new ExtendedBlock(bpid, replica);
         if (idx % 3 == 0) {
           LOG.info("Deliberately removing meta for block " + eb);

+ 0 - 119
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java

@@ -1,119 +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.hdfs.security;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
-import static org.mockito.Mockito.mock;
-
-import java.net.InetSocketAddress;
-import java.security.PrivilegedExceptionAction;
-
-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.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SaslInputStream;
-import org.apache.hadoop.security.SaslRpcClient;
-import org.apache.hadoop.security.SaslRpcServer;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
-import org.junit.Test;
-
-/** Unit tests for using Delegation Token over RPC. */
-public class TestClientProtocolWithDelegationToken {
-  private static final String ADDRESS = "0.0.0.0";
-
-  public static final Log LOG = LogFactory
-      .getLog(TestClientProtocolWithDelegationToken.class);
-
-  private static final Configuration conf;
-  static {
-    conf = new Configuration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-  }
-
-  static {
-    GenericTestUtils.setLogLevel(Client.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(Server.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(SaslRpcClient.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(SaslRpcServer.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(SaslInputStream.LOG, Level.ALL);
-  }
-
-  @Test
-  public void testDelegationTokenRpc() throws Exception {
-    ClientProtocol mockNN = mock(ClientProtocol.class);
-    FSNamesystem mockNameSys = mock(FSNamesystem.class);
-
-    DelegationTokenSecretManager sm = new DelegationTokenSecretManager(
-        DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT,
-        DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT,
-        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT,
-        3600000, mockNameSys);
-    sm.startThreads();
-    final Server server = new RPC.Builder(conf)
-        .setProtocol(ClientProtocol.class).setInstance(mockNN)
-        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
-        .setSecretManager(sm).build();
-    
-    server.start();
-
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    String user = current.getUserName();
-    Text owner = new Text(user);
-    DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner, owner, null);
-    Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>(
-        dtId, sm);
-    SecurityUtil.setTokenService(token, addr);
-    LOG.info("Service for token is " + token.getService());
-    current.addToken(token);
-    current.doAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        ClientProtocol proxy = null;
-        try {
-          proxy = RPC.getProxy(ClientProtocol.class,
-              ClientProtocol.versionID, addr, conf);
-          proxy.getServerDefaults();
-        } finally {
-          server.stop();
-          if (proxy != null) {
-            RPC.stopProxy(proxy);
-          }
-        }
-        return null;
-      }
-    });
-  }
-
-}

+ 26 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -132,7 +132,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   // information about a single block
-  private class BInfo implements ReplicaInPipelineInterface {
+  private class BInfo implements ReplicaInPipeline {
     final Block theBlock;
     private boolean finalized = false; // if not finalized => ongoing creation
     SimulatedOutputStream oStream = null;
@@ -330,6 +330,28 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     public boolean isOnTransientStorage() {
       return false;
     }
+
+    @Override
+    public ReplicaInfo getReplicaInfo() {
+      return null;
+    }
+
+    @Override
+    public void setWriter(Thread writer) {
+    }
+
+    @Override
+    public void interruptThread() {
+    }
+
+    @Override
+    public boolean attemptToSetWriter(Thread prevWriter, Thread newWriter) {
+      return false;
+    }
+
+    @Override
+    public void stopWriter(long xceiverStopTimeout) throws IOException {
+    }
   }
   
   /**
@@ -1228,7 +1250,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public ReplicaInPipelineInterface convertTemporaryToRbw(ExtendedBlock temporary)
+  public ReplicaInPipeline convertTemporaryToRbw(ExtendedBlock temporary)
       throws IOException {
     final Map<Block, BInfo> map = blockMap.get(temporary.getBlockPoolId());
     if (map == null) {
@@ -1302,12 +1324,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public List<FinalizedReplica> getFinalizedBlocks(String bpid) {
+  public List<ReplicaInfo> getFinalizedBlocks(String bpid) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid) {
+  public List<ReplicaInfo> getFinalizedBlocksOnPersistentStorage(String bpid) {
     throw new UnsupportedOperationException();
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

@@ -116,7 +116,7 @@ public class TestBPOfferService {
     File dnDataDir = new File(new File(TEST_BUILD_DATA, "dfs"), "data");
     conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
     Mockito.doReturn(conf).when(mockDn).getConf();
-    Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
+    Mockito.doReturn(new DNConf(mockDn)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))
     .when(mockDn).getMetrics();
 
@@ -338,7 +338,7 @@ public class TestBPOfferService {
       new File(TEST_BUILD_DATA, "testBPInitErrorHandling"), "data");
     conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
     Mockito.doReturn(conf).when(mockDn).getConf();
-    Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
+    Mockito.doReturn(new DNConf(mockDn)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn")).
       when(mockDn).getMetrics();
     final AtomicInteger count = new AtomicInteger();

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java

@@ -21,6 +21,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 import java.io.File;
 import java.util.Random;
@@ -105,7 +106,10 @@ public class TestBlockPoolSliceStorage {
 
     LOG.info("Got subdir " + blockFileSubdir);
     LOG.info("Generated file path " + testFilePath);
-    assertThat(storage.getTrashDirectory(new File(testFilePath)), is(expectedTrashPath));
+
+    ReplicaInfo info = Mockito.mock(ReplicaInfo.class);
+    Mockito.when(info.getBlockURI()).thenReturn(new File(testFilePath).toURI());
+    assertThat(storage.getTrashDirectory(info), is(expectedTrashPath));
   }
 
   /*

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

@@ -667,7 +667,7 @@ public class TestBlockRecovery {
     if(LOG.isDebugEnabled()) {
       LOG.debug("Running " + GenericTestUtils.getMethodName());
     }
-    ReplicaInPipelineInterface replicaInfo = dn.data.createRbw(
+    ReplicaInPipeline replicaInfo = dn.data.createRbw(
         StorageType.DEFAULT, block, false).getReplica();
     ReplicaOutputStreams streams = null;
     try {

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 /**
  * Ensure that the DataNode correctly handles rolling upgrade
@@ -114,8 +115,11 @@ public class TestDataNodeRollingUpgrade {
   }
 
   private File getTrashFileForBlock(File blockFile, boolean exists) {
+
+    ReplicaInfo info = Mockito.mock(ReplicaInfo.class);
+    Mockito.when(info.getBlockURI()).thenReturn(blockFile.toURI());
     File trashFile = new File(
-        dn0.getStorage().getTrashDirectoryForBlockFile(blockPoolId, blockFile));
+        dn0.getStorage().getTrashDirectoryForReplica(blockPoolId, info));
     assertEquals(exists, trashFile.exists());
     return trashFile;
   }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java

@@ -171,12 +171,13 @@ public class TestDataXceiverLazyPersistHint {
     conf.setBoolean(
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST,
         nonLocalLazyPersist == NonLocalLazyPersist.ALLOWED);
-    DNConf dnConf = new DNConf(conf);
+
     DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
     DataNodeMetrics mockMetrics = mock(DataNodeMetrics.class);
     DataNode mockDn = mock(DataNode.class);
-    when(mockDn.getDnConf()).thenReturn(dnConf);
     when(mockDn.getConf()).thenReturn(conf);
+    DNConf dnConf = new DNConf(mockDn);
+    when(mockDn.getDnConf()).thenReturn(dnConf);
     when(mockDn.getMetrics()).thenReturn(mockMetrics);
     when(mockDn.getDNRegistrationForBP("Dummy-pool")).thenReturn(mockDnReg);
     return mockDn;

+ 8 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -116,8 +116,8 @@ public class TestDirectoryScanner {
   private long truncateBlockFile() throws IOException {
     try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
       for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
-        File f = b.getBlockFile();
-        File mf = b.getMetaFile();
+        File f = new File(b.getBlockURI());
+        File mf = new File(b.getMetadataURI());
         // Truncate a block file that has a corresponding metadata file
         if (f.exists() && f.length() != 0 && mf.exists()) {
           FileOutputStream s = null;
@@ -141,8 +141,8 @@ public class TestDirectoryScanner {
   private long deleteBlockFile() {
     try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
       for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
-        File f = b.getBlockFile();
-        File mf = b.getMetaFile();
+        File f = new File(b.getBlockURI());
+        File mf = new File(b.getMetadataURI());
         // Delete a block file that has corresponding metadata file
         if (f.exists() && mf.exists() && f.delete()) {
           LOG.info("Deleting block file " + f.getAbsolutePath());
@@ -157,10 +157,9 @@ public class TestDirectoryScanner {
   private long deleteMetaFile() {
     try(AutoCloseableLock lock = fds.acquireDatasetLock()) {
       for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
-        File file = b.getMetaFile();
         // Delete a metadata file
-        if (file.exists() && file.delete()) {
-          LOG.info("Deleting metadata file " + file.getAbsolutePath());
+        if (b.metadataExists() && b.deleteMetadata()) {
+          LOG.info("Deleting metadata " + b.getMetadataURI());
           return b.getBlockId();
         }
       }
@@ -184,8 +183,8 @@ public class TestDirectoryScanner {
           }
 
           // Volume without a copy of the block. Make a copy now.
-          File sourceBlock = b.getBlockFile();
-          File sourceMeta = b.getMetaFile();
+          File sourceBlock = new File(b.getBlockURI());
+          File sourceMeta = new File(b.getMetadataURI());
           String sourceRoot = b.getVolume().getBasePath();
           String destRoot = v.getBasePath();
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java

@@ -80,7 +80,7 @@ public class TestSimulatedFSDataset {
       ExtendedBlock b = new ExtendedBlock(bpid, blkID, 0, 0);
       // we pass expected len as zero, - fsdataset should use the sizeof actual
       // data written
-      ReplicaInPipelineInterface bInfo = fsdataset.createRbw(
+      ReplicaInPipeline bInfo = fsdataset.createRbw(
           StorageType.DEFAULT, b, false).getReplica();
       ReplicaOutputStreams out = bInfo.createStreams(true,
           DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java

@@ -57,7 +57,7 @@ public class TestTransferRbw {
       String bpid) throws InterruptedException {
     return (ReplicaBeingWritten)getReplica(datanode, bpid, ReplicaState.RBW);
   }
-  private static ReplicaInPipeline getReplica(final DataNode datanode,
+  private static LocalReplicaInPipeline getReplica(final DataNode datanode,
       final String bpid, final ReplicaState expectedState) throws InterruptedException {
     final Collection<ReplicaInfo> replicas = FsDatasetTestUtil.getReplicas(
         datanode.getFSDataset(), bpid);
@@ -68,7 +68,7 @@ public class TestTransferRbw {
     Assert.assertEquals(1, replicas.size());
     final ReplicaInfo r = replicas.iterator().next();
     Assert.assertEquals(expectedState, r.getState());
-    return (ReplicaInPipeline)r;
+    return (LocalReplicaInPipeline)r;
   }
 
   @Test

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java

@@ -87,12 +87,12 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public List<FinalizedReplica> getFinalizedBlocks(String bpid) {
+  public List<ReplicaInfo> getFinalizedBlocks(String bpid) {
     return null;
   }
 
   @Override
-  public List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid) {
+  public List<ReplicaInfo> getFinalizedBlocksOnPersistentStorage(String bpid) {
     return null;
   }
 
@@ -159,7 +159,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public ReplicaInPipelineInterface convertTemporaryToRbw(
+  public ReplicaInPipeline convertTemporaryToRbw(
       ExtendedBlock temporary) throws IOException {
     return new ExternalReplicaInPipeline();
   }

+ 24 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java

@@ -23,11 +23,12 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.ChunkChecksum;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.util.DataChecksum;
 
-public class ExternalReplicaInPipeline implements ReplicaInPipelineInterface {
+public class ExternalReplicaInPipeline implements ReplicaInPipeline {
 
   @Override
   public void setNumBytes(long bytesReceived) {
@@ -105,4 +106,25 @@ public class ExternalReplicaInPipeline implements ReplicaInPipelineInterface {
   public boolean isOnTransientStorage() {
     return false;
   }
+
+  @Override
+  public ReplicaInfo getReplicaInfo() {
+    return null;
+  }
+
+  public void setWriter(Thread writer) {
+  }
+
+  public void stopWriter(long xceiverStopTimeout)
+      throws IOException {
+  }
+
+  @Override
+  public boolean attemptToSetWriter(Thread prevWriter, Thread newWriter) {
+    return false;
+  }
+
+  @Override
+  public void interruptThread() {
+  }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/TestExternalDataset.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.hdfs.server.datanode.extdataset;
 
 import org.apache.hadoop.hdfs.server.datanode.Replica;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.junit.Test;
@@ -75,7 +75,7 @@ public class TestExternalDataset {
    */
   @Test
   public void testInstantiateReplicaInPipeline() throws Throwable {
-    ReplicaInPipelineInterface inst = new ExternalReplicaInPipeline();
+    ReplicaInPipeline inst = new ExternalReplicaInPipeline();
   }
 
   /**

+ 21 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java

@@ -26,8 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@@ -35,11 +35,11 @@ import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
+import org.apache.hadoop.hdfs.server.datanode.LocalReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -205,8 +205,8 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
     dataset = (FsDatasetImpl) datanode.getFSDataset();
   }
 
-  private File getBlockFile(ExtendedBlock eb) throws IOException {
-    return dataset.getBlockFile(eb.getBlockPoolId(), eb.getBlockId());
+  private ReplicaInfo getBlockFile(ExtendedBlock eb) throws IOException {
+    return dataset.getReplicaInfo(eb);
   }
 
   /**
@@ -217,8 +217,8 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
       throws ReplicaNotFoundException {
     File blockFile;
     try {
-       blockFile = dataset.getBlockFile(
-           block.getBlockPoolId(), block.getBlockId());
+      ReplicaInfo r = dataset.getReplicaInfo(block);
+      blockFile = new File(r.getBlockURI());
     } catch (IOException e) {
       LOG.error("Block file for " + block + " does not existed:", e);
       throw new ReplicaNotFoundException(block);
@@ -240,7 +240,7 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
   public Replica createFinalizedReplica(FsVolumeSpi volume, ExtendedBlock block)
       throws IOException {
     FsVolumeImpl vol = (FsVolumeImpl) volume;
-    ReplicaInfo info = new FinalizedReplica(block.getLocalBlock(), vol,
+    FinalizedReplica info = new FinalizedReplica(block.getLocalBlock(), vol,
         vol.getCurrentDir().getParentFile());
     dataset.volumeMap.add(block.getBlockPoolId(), info);
     info.getBlockFile().createNewFile();
@@ -260,7 +260,7 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
   public Replica createReplicaInPipeline(
       FsVolumeSpi volume, ExtendedBlock block) throws IOException {
     FsVolumeImpl vol = (FsVolumeImpl) volume;
-    ReplicaInPipeline rip = new ReplicaInPipeline(
+    LocalReplicaInPipeline rip = new LocalReplicaInPipeline(
         block.getBlockId(), block.getGenerationStamp(), volume,
         vol.createTmpFile(
             block.getBlockPoolId(), block.getLocalBlock()).getParentFile(),
@@ -305,9 +305,11 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
     FsVolumeImpl vol = (FsVolumeImpl) volume;
     final String bpid = eb.getBlockPoolId();
     final Block block = eb.getLocalBlock();
-    ReplicaWaitingToBeRecovered rwbr =
-        new ReplicaWaitingToBeRecovered(eb.getLocalBlock(), volume,
-            vol.createRbwFile(bpid, block).getParentFile());
+    ReplicaInfo rwbr = new ReplicaBuilder(ReplicaState.RWR)
+        .setBlock(eb.getLocalBlock())
+        .setFsVolume(volume)
+        .setDirectoryToUse(vol.createRbwFile(bpid, block).getParentFile())
+        .build();
     dataset.volumeMap.add(bpid, rwbr);
     return rwbr;
   }
@@ -354,6 +356,7 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
             "Meta file " + metaFile + " already exists."
         );
       }
+      dataset.volumeMap.add(block.getBlockPoolId(), finalized);
     }
   }
 
@@ -379,25 +382,21 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
 
   @Override
   public long getStoredDataLength(ExtendedBlock block) throws IOException {
-    File f = getBlockFile(block);
-    try (RandomAccessFile raf = new RandomAccessFile(f, "r")) {
-      return raf.length();
-    }
+    ReplicaInfo r = getBlockFile(block);
+    return r.getBlockDataLength();
   }
 
   @Override
   public long getStoredGenerationStamp(ExtendedBlock block) throws IOException {
-    File f = getBlockFile(block);
-    File dir = f.getParentFile();
-    File[] files = FileUtil.listFiles(dir);
-    return FsDatasetUtil.getGenerationStampFromFile(files, f);
+    ReplicaInfo r = getBlockFile(block);
+    return r.getGenerationStamp();
   }
 
   @Override
   public void changeStoredGenerationStamp(
       ExtendedBlock block, long newGenStamp) throws IOException {
-    File blockFile =
-        dataset.getBlockFile(block.getBlockPoolId(), block.getBlockId());
+    ReplicaInfo r = dataset.getReplicaInfo(block);
+    File blockFile = new File(r.getBlockURI());
     File metaFile = FsDatasetUtil.findMetaFile(blockFile);
     File newMetaFile = new File(
         DatanodeUtil.getMetaName(blockFile.getAbsolutePath(), newGenStamp));

+ 15 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java

@@ -24,16 +24,16 @@ import java.nio.channels.FileChannel;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
 import java.util.Collection;
-import java.util.Random;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.LocalReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
-import org.apache.hadoop.io.IOUtils;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
@@ -41,12 +41,21 @@ import static org.junit.Assert.fail;
 public class FsDatasetTestUtil {
 
   public static File getFile(FsDatasetSpi<?> fsd, String bpid, long bid) {
-    return ((FsDatasetImpl)fsd).getFile(bpid, bid, false);
+    ReplicaInfo r;
+    try {
+      r = ((FsDatasetImpl)fsd).getReplicaInfo(bpid, bid);
+      return new File(r.getBlockURI());
+    } catch (ReplicaNotFoundException e) {
+      FsDatasetImpl.LOG.warn(String.format(
+          "Replica with id %d was not found in block pool %s.", bid, bpid), e);
+    }
+    return null;
   }
 
   public static File getBlockFile(FsDatasetSpi<?> fsd, String bpid, Block b
       ) throws IOException {
-    return ((FsDatasetImpl)fsd).getBlockFile(bpid, b.getBlockId());
+    ReplicaInfo r = ((FsDatasetImpl)fsd).getReplicaInfo(bpid, b.getBlockId());
+    return new File(r.getBlockURI());
   }
 
   public static File getMetaFile(FsDatasetSpi<?> fsd, String bpid, Block b)
@@ -57,7 +66,8 @@ public class FsDatasetTestUtil {
 
   public static boolean breakHardlinksIfNeeded(FsDatasetSpi<?> fsd,
       ExtendedBlock block) throws IOException {
-    final ReplicaInfo info = ((FsDatasetImpl)fsd).getReplicaInfo(block);
+    final LocalReplica info =
+        (LocalReplica) ((FsDatasetImpl)fsd).getReplicaInfo(block);
     return info.breakHardLinksIfNeeded();
   }
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -163,11 +163,11 @@ public class TestFsDatasetImpl {
     storage = mock(DataStorage.class);
     this.conf = new Configuration();
     this.conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
-    final DNConf dnConf = new DNConf(conf);
 
     when(datanode.getConf()).thenReturn(conf);
+    final DNConf dnConf = new DNConf(datanode);
     when(datanode.getDnConf()).thenReturn(dnConf);
-    final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
+    final BlockScanner disabledBlockScanner = new BlockScanner(datanode);
     when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
     final ShortCircuitRegistry shortCircuitRegistry =
         new ShortCircuitRegistry(conf);
@@ -326,7 +326,7 @@ public class TestFsDatasetImpl {
     RoundRobinVolumeChoosingPolicy<FsVolumeImpl> blockChooser =
         new RoundRobinVolumeChoosingPolicy<>();
     conf.setLong(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
-    final BlockScanner blockScanner = new BlockScanner(datanode, conf);
+    final BlockScanner blockScanner = new BlockScanner(datanode);
     final FsVolumeList volumeList = new FsVolumeList(
         Collections.<VolumeFailureInfo>emptyList(), blockScanner, blockChooser);
     final List<FsVolumeImpl> oldVolumes = new ArrayList<>();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -483,7 +483,7 @@ public class TestWriteToReplica {
     long newGenStamp = blocks[NON_EXISTENT].getGenerationStamp() * 10;
     blocks[NON_EXISTENT].setGenerationStamp(newGenStamp);
     try {
-      ReplicaInPipelineInterface replicaInfo =
+      ReplicaInPipeline replicaInfo =
           dataSet.createTemporary(StorageType.DEFAULT, blocks[NON_EXISTENT]).getReplica();
       Assert.assertTrue(replicaInfo.getGenerationStamp() == newGenStamp);
       Assert.assertTrue(

+ 26 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java

@@ -43,6 +43,8 @@ import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -140,6 +142,27 @@ public class TestDiskBalancerCommand {
             containsString("9 volumes with node data density 1.97"))));
   }
 
+  /**
+   * This test simulates DiskBalancerCLI Report command run from a shell
+   * with a generic option 'fs'.
+   * @throws Exception
+   */
+  @Test(timeout = 60000)
+  public void testReportWithGenericOptionFS() throws Exception {
+    final String topReportArg = "5";
+    final String reportArgs = String.format("-%s file:%s -%s -%s %s",
+        "fs", clusterJson.getPath(),
+        REPORT, "top", topReportArg);
+    final String cmdLine = String.format("%s", reportArgs);
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(outputs.get(0), containsString("Processing report command"));
+    assertThat(outputs.get(1),
+        is(allOf(containsString("Reporting top"), containsString(topReportArg),
+            containsString(
+                "DataNode(s) benefiting from running DiskBalancer"))));
+  }
+
   /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
   @Test(timeout = 60000)
   public void testReportMoreThanTotal() throws Exception {
@@ -389,11 +412,11 @@ public class TestDiskBalancerCommand {
   private List<String> runCommandInternal(final String cmdLine) throws
       Exception {
     String[] cmds = StringUtils.split(cmdLine, ' ');
-    DiskBalancerCLI db = new DiskBalancerCLI(conf);
-
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
-    db.run(cmds, out);
+
+    Tool diskBalancerTool = new DiskBalancerCLI(conf, out);
+    ToolRunner.run(conf, diskBalancerTool, cmds);
 
     Scanner scanner = new Scanner(bufOut.toString());
     List<String> outputs = Lists.newArrayList();

+ 0 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
@@ -98,8 +97,6 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
     BlockingService refreshHSAdminProtocolService = HSAdminRefreshProtocolService
         .newReflectiveBlockingService(refreshHSAdminProtocolXlator);
 
-    WritableRpcEngine.ensureInitialized();
-
     clientRpcAddress = conf.getSocketAddr(
         JHAdminConfig.MR_HISTORY_BIND_HOST,
         JHAdminConfig.JHS_ADMIN_ADDRESS,

+ 124 - 35
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
@@ -106,9 +107,14 @@ public class TestDFSIO implements Tool {
                     " [-nrFiles N]" +
                     " [-size Size[B|KB|MB|GB|TB]]" +
                     " [-resFile resultFileName] [-bufferSize Bytes]" +
-                    " [-storagePolicy storagePolicyName]";
+                    " [-storagePolicy storagePolicyName]" +
+                    " [-erasureCodePolicy erasureCodePolicyName]";
 
   private Configuration config;
+  private static final String STORAGE_POLICY_NAME_KEY =
+      "test.io.block.storage.policy";
+  private static final String ERASURE_CODE_POLICY_NAME_KEY =
+      "test.io.erasure.code.policy";
 
   static{
     Configuration.addDefaultResource("hdfs-default.xml");
@@ -211,9 +217,9 @@ public class TestDFSIO implements Tool {
     bench = new TestDFSIO();
     bench.getConf().setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     cluster = new MiniDFSCluster.Builder(bench.getConf())
-                                .numDataNodes(2)
-                                .format(true)
-                                .build();
+        .numDataNodes(2)
+        .format(true)
+        .build();
     FileSystem fs = cluster.getFileSystem();
     bench.createControlFile(fs, DEFAULT_NR_BYTES, DEFAULT_NR_FILES);
 
@@ -356,7 +362,7 @@ public class TestDFSIO implements Tool {
             ReflectionUtils.newInstance(codec, getConf());
       }
 
-      blockStoragePolicy = getConf().get("test.io.block.storage.policy", null);
+      blockStoragePolicy = getConf().get(STORAGE_POLICY_NAME_KEY, null);
     }
 
     @Override // IOMapperBase
@@ -388,9 +394,10 @@ public class TestDFSIO implements Tool {
    */
   public static class WriteMapper extends IOStatMapper {
 
-    public WriteMapper() { 
-      for(int i=0; i < bufferSize; i++)
-        buffer[i] = (byte)('0' + i % 50);
+    public WriteMapper() {
+      for (int i = 0; i < bufferSize; i++) {
+        buffer[i] = (byte) ('0' + i % 50);
+      }
     }
 
     @Override // IOMapperBase
@@ -431,6 +438,9 @@ public class TestDFSIO implements Tool {
     fs.delete(getDataDir(config), true);
     fs.delete(writeDir, true);
     long tStart = System.currentTimeMillis();
+    if (isECEnabled()) {
+      createAndEnableECOnPath(fs, getDataDir(config));
+    }
     runIOTest(WriteMapper.class, writeDir);
     long execTime = System.currentTimeMillis() - tStart;
     return execTime;
@@ -734,6 +744,7 @@ public class TestDFSIO implements Tool {
     TestType testType = null;
     int bufferSize = DEFAULT_BUFFER_SIZE;
     long nrBytes = 1*MEGA;
+    String erasureCodePolicyName = null;
     int nrFiles = 1;
     long skipSize = 0;
     String resFileName = DEFAULT_RES_FILE_NAME;
@@ -785,26 +796,31 @@ public class TestDFSIO implements Tool {
         resFileName = args[++i];
       } else if (args[i].equalsIgnoreCase("-storagePolicy")) {
         storagePolicy = args[++i];
+      } else if (args[i].equalsIgnoreCase("-erasureCodePolicy")) {
+        erasureCodePolicyName = args[++i];
       } else {
         System.err.println("Illegal argument: " + args[i]);
         return -1;
       }
     }
-    if(testType == null)
+    if (testType == null) {
       return -1;
-    if(testType == TestType.TEST_TYPE_READ_BACKWARD)
+    }
+    if (testType == TestType.TEST_TYPE_READ_BACKWARD) {
       skipSize = -bufferSize;
-    else if(testType == TestType.TEST_TYPE_READ_SKIP && skipSize == 0)
+    } else if (testType == TestType.TEST_TYPE_READ_SKIP && skipSize == 0) {
       skipSize = bufferSize;
+    }
 
     LOG.info("nrFiles = " + nrFiles);
     LOG.info("nrBytes (MB) = " + toMB(nrBytes));
     LOG.info("bufferSize = " + bufferSize);
-    if(skipSize > 0)
+    if (skipSize > 0) {
       LOG.info("skipSize = " + skipSize);
+    }
     LOG.info("baseDir = " + getBaseDir(config));
     
-    if(compressionClass != null) {
+    if (compressionClass != null) {
       config.set("test.io.compression.class", compressionClass);
       LOG.info("compressionClass = " + compressionClass);
     }
@@ -813,31 +829,16 @@ public class TestDFSIO implements Tool {
     config.setLong("test.io.skip.size", skipSize);
     FileSystem fs = FileSystem.get(config);
 
-    if (storagePolicy != null) {
-      boolean isValid = false;
-      Collection<BlockStoragePolicy> storagePolicies =
-          ((DistributedFileSystem) fs).getAllStoragePolicies();
-      try {
-        for (BlockStoragePolicy policy : storagePolicies) {
-          if (policy.getName().equals(storagePolicy)) {
-            isValid = true;
-            break;
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException("Get block storage policies error: ", e);
-      }
-      if (!isValid) {
-        System.out.println("Invalid block storage policy: " + storagePolicy);
-        System.out.println("Current supported storage policy list: ");
-        for (BlockStoragePolicy policy : storagePolicies) {
-          System.out.println(policy.getName());
-        }
+    if (erasureCodePolicyName != null) {
+      if (!checkErasureCodePolicy(erasureCodePolicyName, fs, testType)) {
         return -1;
       }
+    }
 
-      config.set("test.io.block.storage.policy", storagePolicy);
-      LOG.info("storagePolicy = " + storagePolicy);
+    if (storagePolicy != null) {
+      if (!checkStoragePolicy(storagePolicy, fs)) {
+        return -1;
+      }
     }
 
     if (isSequential) {
@@ -908,6 +909,94 @@ public class TestDFSIO implements Tool {
     return ((float)bytes)/MEGA;
   }
 
+  private boolean checkErasureCodePolicy(String erasureCodePolicyName,
+      FileSystem fs, TestType testType) throws IOException {
+    Collection<ErasureCodingPolicy> list =
+        ((DistributedFileSystem) fs).getAllErasureCodingPolicies();
+    boolean isValid = false;
+    for (ErasureCodingPolicy ec : list) {
+      if (erasureCodePolicyName.equals(ec.getName())) {
+        isValid = true;
+        break;
+      }
+    }
+
+    if (!isValid) {
+      System.out.println("Invalid erasure code policy: " +
+          erasureCodePolicyName);
+      System.out.println("Current supported erasure code policy list: ");
+      for (ErasureCodingPolicy ec : list) {
+        System.out.println(ec.getName());
+      }
+      return false;
+    }
+
+    if (testType == TestType.TEST_TYPE_APPEND ||
+        testType == TestType.TEST_TYPE_TRUNCATE) {
+      System.out.println("So far append or truncate operation" +
+          " does not support erasureCodePolicy");
+      return false;
+    }
+
+    config.set(ERASURE_CODE_POLICY_NAME_KEY, erasureCodePolicyName);
+    LOG.info("erasureCodePolicy = " + erasureCodePolicyName);
+    return true;
+  }
+
+  private boolean checkStoragePolicy(String storagePolicy, FileSystem fs)
+      throws IOException {
+    boolean isValid = false;
+    Collection<BlockStoragePolicy> storagePolicies =
+        ((DistributedFileSystem) fs).getAllStoragePolicies();
+    try {
+      for (BlockStoragePolicy policy : storagePolicies) {
+        if (policy.getName().equals(storagePolicy)) {
+          isValid = true;
+          break;
+        }
+      }
+    } catch (Exception e) {
+      throw new IOException("Get block storage policies error: ", e);
+    }
+
+    if (!isValid) {
+      System.out.println("Invalid block storage policy: " + storagePolicy);
+      System.out.println("Current supported storage policy list: ");
+      for (BlockStoragePolicy policy : storagePolicies) {
+        System.out.println(policy.getName());
+      }
+      return false;
+    }
+
+    config.set(STORAGE_POLICY_NAME_KEY, storagePolicy);
+    LOG.info("storagePolicy = " + storagePolicy);
+    return true;
+  }
+
+  private boolean isECEnabled() {
+    String erasureCodePolicyName =
+        getConf().get(ERASURE_CODE_POLICY_NAME_KEY, null);
+    return erasureCodePolicyName != null ? true : false;
+  }
+
+  void createAndEnableECOnPath(FileSystem fs, Path path)
+      throws IOException {
+    String erasureCodePolicyName =
+        getConf().get(ERASURE_CODE_POLICY_NAME_KEY, null);
+
+    fs.mkdirs(path);
+    Collection<ErasureCodingPolicy> list =
+        ((DistributedFileSystem) fs).getAllErasureCodingPolicies();
+    for (ErasureCodingPolicy ec : list) {
+      if (erasureCodePolicyName.equals(ec.getName())) {
+        ((DistributedFileSystem) fs).setErasureCodingPolicy(path, ec);
+        LOG.info("enable erasureCodePolicy = " + erasureCodePolicyName  +
+            " on " + path.toString());
+        break;
+      }
+    }
+  }
+
   private void analyzeResult( FileSystem fs,
                               TestType testType,
                               long execTime,

+ 16 - 0
hadoop-tools/hadoop-openstack/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem

@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java

@@ -249,6 +249,7 @@ public class RMAdminCLI extends HAAdmin {
         " [-replaceLabelsOnNode <\"node1[:port]=label1,label2" +
         " node2[:port]=label1\">]" +
         " [-directlyAccessNodeLabelStore]" +
+        " [-refreshClusterMaxPriority]" +
         " [-updateNodeResource [NodeID] [MemSize] [vCores]" +
         " ([OvercommitTimeout])");
     if (isHAEnabled) {

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java

@@ -471,9 +471,9 @@ public class TestRMAdminCLI {
               "[-removeFromClusterNodeLabels <label1,label2,label3>] " +
               "[-replaceLabelsOnNode " +
               "<\"node1[:port]=label1,label2 node2[:port]=label1\">] " +
-              "[-directlyAccessNodeLabelStore] [-updateNodeResource " +
-              "[NodeID] [MemSize] [vCores] ([OvercommitTimeout]) " +
-              "[-help [cmd]]"));
+              "[-directlyAccessNodeLabelStore] [-refreshClusterMaxPriority] " +
+              "[-updateNodeResource [NodeID] [MemSize] [vCores] " +
+              "([OvercommitTimeout]) [-help [cmd]]"));
       assertTrue(dataOut
           .toString()
           .contains(
@@ -565,7 +565,9 @@ public class TestRMAdminCLI {
                   + "label2(exclusive=false),label3\">]"
               + " [-removeFromClusterNodeLabels <label1,label2,label3>] [-replaceLabelsOnNode "
               + "<\"node1[:port]=label1,label2 node2[:port]=label1\">] [-directlyAccessNodeLabelStore] "
-              + "[-updateNodeResource [NodeID] [MemSize] [vCores] ([OvercommitTimeout]) "
+              + "[-refreshClusterMaxPriority] "
+              + "[-updateNodeResource [NodeID] [MemSize] [vCores] "
+              + "([OvercommitTimeout]) "
               + "[-transitionToActive [--forceactive] <serviceId>] "
               + "[-transitionToStandby <serviceId>] "
               + "[-getServiceState <serviceId>] [-checkHealth <serviceId>] [-help [cmd]]";

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